You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2019/05/23 16:18:45 UTC

[flink] 05/08: [hotfix][network, tests] Add new unit test for RemoteInputChannel#retriggerSubpartitionRequest

This is an automated email from the ASF dual-hosted git repository.

trohrmann pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9f2338cc1999cb57264451ab17d4325beda065c3
Author: Zhijiang <wa...@aliyun.com>
AuthorDate: Mon May 20 19:04:45 2019 +0800

    [hotfix][network,tests] Add new unit test for RemoteInputChannel#retriggerSubpartitionRequest
    
    It is necessary for flip1 to make sure the PartitionNotFoundException would be setted on the RemoteInputChannel while retriggering partition request,
    so a new unit test is added to cover this case.
---
 .../io/network/TestingConnectionManager.java       | 50 ++++++++++++++++++++++
 .../io/network/TestingPartitionRequestClient.java  | 50 ++++++++++++++++++++++
 .../partition/consumer/RemoteInputChannelTest.java | 25 +++++++++++
 3 files changed, 125 insertions(+)

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TestingConnectionManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TestingConnectionManager.java
new file mode 100644
index 0000000..822314d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TestingConnectionManager.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network;
+
+/**
+ * A dummy implementation of the {@link ConnectionManager} which is mainly used for creating
+ * {@link PartitionRequestClient} instance in tests.
+ */
+public class TestingConnectionManager implements ConnectionManager {
+
+	@Override
+	public void start() {}
+
+	@Override
+	public PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) {
+		return new TestingPartitionRequestClient();
+	}
+
+	@Override
+	public void closeOpenChannelConnections(ConnectionID connectionId) {}
+
+	@Override
+	public int getNumberOfActiveConnections() {
+		return 0;
+	}
+
+	@Override
+	public int getDataPort() {
+		return -1;
+	}
+
+	@Override
+	public void shutdown() {}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TestingPartitionRequestClient.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TestingPartitionRequestClient.java
new file mode 100644
index 0000000..68abf63
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TestingPartitionRequestClient.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network;
+
+import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
+
+/**
+ * A dummy implementation of the {@link PartitionRequestClient} instance which is mainly used
+ * for tests to avoid mock.
+ */
+public class TestingPartitionRequestClient implements PartitionRequestClient {
+
+	@Override
+	public void requestSubpartition(
+		ResultPartitionID partitionId,
+		int subpartitionIndex,
+		RemoteInputChannel inputChannel,
+		int delayMs) {
+	}
+
+	@Override
+	public void notifyCreditAvailable(RemoteInputChannel inputChannel) {
+	}
+
+	@Override
+	public void sendTaskEvent(ResultPartitionID partitionId, TaskEvent event, RemoteInputChannel inputChannel) {
+	}
+
+	@Override
+	public void close(RemoteInputChannel inputChannel) {
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
index 6c03106..24d256e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java
@@ -23,11 +23,13 @@ import org.apache.flink.runtime.execution.CancelTaskException;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.ConnectionManager;
 import org.apache.flink.runtime.io.network.PartitionRequestClient;
+import org.apache.flink.runtime.io.network.TestingConnectionManager;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferListener.NotificationResult;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils;
+import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
 import org.apache.flink.runtime.io.network.partition.ProducerFailedException;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.io.network.util.TestBufferFactory;
@@ -51,6 +53,7 @@ import java.util.concurrent.Future;
 import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.hasProperty;
+import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -991,6 +994,28 @@ public class RemoteInputChannelTest {
 		}
 	}
 
+	/**
+	 * Tests that {@link RemoteInputChannel#retriggerSubpartitionRequest(int)} would throw
+	 * the {@link PartitionNotFoundException} if backoff is 0.
+	 */
+	@Test
+	public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exception {
+		final RemoteInputChannel inputChannel = InputChannelTestUtils.createRemoteInputChannel(
+			createSingleInputGate(1), 0, new TestingConnectionManager());
+
+		// Request partition to initialize client to avoid illegal state after retriggering partition
+		inputChannel.requestSubpartition(0);
+		// The default backoff is 0 then it would set PartitionNotFoundException on this channel
+		inputChannel.retriggerSubpartitionRequest(0);
+		try {
+			inputChannel.checkError();
+
+			fail("Should throw a PartitionNotFoundException.");
+		} catch (PartitionNotFoundException notFound) {
+			assertThat(inputChannel.getPartitionId(), is(notFound.getPartitionId()));
+		}
+	}
+
 	// ---------------------------------------------------------------------------------------------
 
 	private RemoteInputChannel createRemoteInputChannel(SingleInputGate inputGate)