You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by zh...@apache.org on 2020/06/03 02:32:03 UTC

[flink] branch master updated: [FLINK-17992][checkpointing] Exception from RemoteInputChannel#onBuffer should not fail the whole NetworkClientHandler

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 371f3de  [FLINK-17992][checkpointing] Exception from RemoteInputChannel#onBuffer should not fail the whole NetworkClientHandler
371f3de is described below

commit 371f3de5371afb78d465315098bebec6ed36656b
Author: Zhijiang <wa...@aliyun.com>
AuthorDate: Thu May 28 09:00:34 2020 +0200

    [FLINK-17992][checkpointing] Exception from RemoteInputChannel#onBuffer should not fail the whole NetworkClientHandler
    
    RemoteInputChannel#onBuffer is invoked by CreditBasedPartitionRequestClientHandler while receiving and decoding the network data. #onBuffer can
    throw exceptions which would tag the error in client handler and fail all the added input channels inside handler. Then it would cause a tricky
    potential issue as following.
    
    If the RemoteInputChannel is canceling by canceler thread, then the task thread might exit early than canceler thread terminate. That means the
    PartitionRequestClient might not be closed (triggered by canceler thread) while the new task attempt is already deployed into the same TaskManager.
    Therefore the new task might reuse the previous PartitionRequestClient while requesting partitions, but note that the respective client handler was
    already tagged an error before during above RemoteInputChannel#onBuffer, to cause the next round unnecessary failover.
    
    The solution is to only fail the respective task when its internal RemoteInputChannel#onBuffer throws any exceptions instead of failing the whole
    channels inside client handler, then the client is still healthy and can also be reused by other input channels as long as it is not released yet.
---
 .../CreditBasedPartitionRequestClientHandler.java  | 11 +++-
 ...editBasedPartitionRequestClientHandlerTest.java | 71 ++++++++++++++++++++++
 2 files changed, 80 insertions(+), 2 deletions(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java
index 4f3d872..5097c13 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.io.network.NetworkClientHandler;
 import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
 import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
@@ -236,7 +237,8 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap
 	/**
 	 * Checks for an error and rethrows it if one was reported.
 	 */
-	private void checkError() throws IOException {
+	@VisibleForTesting
+	void checkError() throws IOException {
 		final Throwable t = channelError.get();
 
 		if (t != null) {
@@ -264,7 +266,12 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap
 				return;
 			}
 
-			decodeBufferOrEvent(inputChannel, bufferOrEvent);
+			try {
+				decodeBufferOrEvent(inputChannel, bufferOrEvent);
+			} catch (Throwable t) {
+				inputChannel.onError(t);
+			}
+
 
 		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
 			// ---- Error ---------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java
index 9487c1c..8cfa4e4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.io.network.netty;
 
+import org.apache.flink.metrics.SimpleCounter;
 import org.apache.flink.runtime.io.network.ConnectionID;
 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.BufferCompressor;
 import org.apache.flink.runtime.io.network.buffer.BufferDecompressor;
@@ -486,6 +488,47 @@ public class CreditBasedPartitionRequestClientHandlerTest {
 		testReadBufferResponseWithReleasingOrRemovingChannel(true, false);
 	}
 
+	@Test
+	public void testDoNotFailHandlerOnSingleChannelFailure() throws Exception {
+		// Setup
+		final int bufferSize = 1024;
+		final String expectedMessage = "test exception on buffer";
+		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(10, bufferSize, 2);
+		final SingleInputGate inputGate = createSingleInputGate(1, networkBufferPool);
+		final RemoteInputChannel inputChannel = new TestRemoteInputChannelForError(inputGate, expectedMessage);
+		final CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler();
+
+		try {
+			inputGate.setInputChannels(inputChannel);
+			inputGate.assignExclusiveSegments();
+			inputGate.requestPartitions();
+			handler.addInputChannel(inputChannel);
+
+			final BufferResponse bufferResponse = createBufferResponse(
+				TestBufferFactory.createBuffer(bufferSize),
+				0,
+				inputChannel.getInputChannelId(),
+				1,
+				new NetworkBufferAllocator(handler));
+
+			// It will trigger an expected exception from TestRemoteInputChannelForError#onBuffer
+			handler.channelRead(null, bufferResponse);
+
+			// The handler should not be tagged as error for above excepted exception
+			handler.checkError();
+
+			try {
+				// The input channel should be tagged as error and the respective exception is thrown via #getNext
+				inputGate.getNext();
+			} catch (IOException ignored) {
+				assertEquals(expectedMessage, ignored.getMessage());
+			}
+		} finally {
+			// Cleanup
+			releaseResource(inputGate, networkBufferPool);
+		}
+	}
+
 	private void testReadBufferResponseWithReleasingOrRemovingChannel(
 		boolean isRemoved,
 		boolean readBeforeReleasingOrRemoving) throws Exception {
@@ -575,4 +618,32 @@ public class CreditBasedPartitionRequestClientHandlerTest {
 		// Deserialize the bytes to construct the BufferResponse.
 		return BufferResponse.readFrom(serialized, allocator);
 	}
+
+	/**
+	 * The test remote input channel to throw expected exception while calling
+	 * {@link RemoteInputChannel#onBuffer(Buffer, int, int)}.
+	 */
+	private static class TestRemoteInputChannelForError extends RemoteInputChannel {
+		private final String expectedMessage;
+
+		TestRemoteInputChannelForError(SingleInputGate inputGate, String expectedMessage) {
+			super(
+				inputGate,
+				0,
+				new ResultPartitionID(),
+				InputChannelBuilder.STUB_CONNECTION_ID,
+				new TestingConnectionManager(),
+				0,
+				100,
+				new SimpleCounter(),
+				new SimpleCounter());
+			this.expectedMessage = expectedMessage;
+		}
+
+		@Override
+		public void onBuffer(Buffer buffer, int sequenceNumber, int backlog) throws IOException {
+			buffer.recycleBuffer();
+			throw new IOException(expectedMessage);
+		}
+	}
 }