You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by zhijiangW <gi...@git.apache.org> on 2017/08/10 05:57:54 UTC

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

GitHub user zhijiangW opened a pull request:

    https://github.com/apache/flink/pull/4509

    [FLINK-7406][network] Implement Netty receiver incoming pipeline for credit-based

    ## What is the purpose of the change
    
    Currently `PartitionRequestClientHandler` receives and reads BufferResponse from producer. It will request buffer from `BufferPool` for holding the message. If not got, the message is staged temporarily and autoread of channel is set false.
    
    For credit-based mode, `PartitionRequestClientHandler` can always get buffer from `RemoteInputChannel` for reading messages from producer, and update backlog of producer to trigger requests of floating buffers.
    
    This pull request depends on previous two [PRs](https://github.com/apache/flink/pull/4499) whose commits are also included for passing travis, so review the third commit for this pull request change.
    
    In order not to affect the current process and existing cases, we implement a temporary netty handler called `CreditBasedClientHanlder` which will replace the current `PartitionRequestClientHandler` after whole feature codes are submitted.
    
    ## Brief change log
    
      - *Add the backlog of producer in `BufferResponse` message structure*
      - *Netty handler requests buffer from `RemoteInputChannel` directly*
      - *Netty handler updates backlog for `RemoteInputChannel`, and it may trigger requests of floating buffers from `BufferPool`*
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
      - *Modified netty message serialization tests for new added backlog in `BufferResponse`*
      - *Some additional tests and cases would be supplied after the sender logic submitted*
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (not applicable)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zhijiangW/flink FLINK-7406

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4509.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4509
    
----
commit 5c7a27f6fdd215150174c7827cc87b5ea08e01bc
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-07T09:31:17Z

    [FLINK-7378][core]Implement the FixedBufferPool for floating buffers of SingleInputGate

commit f5fb7baeeac3be3719555e37c39d07839d094799
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-09T05:43:56Z

    [FLINK-7394][core]Implement basic InputChannel for credit-based logic

commit d6e35a06e43cf6e7e8d548ebff34dd9877f749bb
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-10T05:29:13Z

    [FLINK-7406][network]Implement Netty receiver incoming pipeline for credit-based

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141896488
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -412,23 +468,35 @@ public void onBuffer(Buffer buffer, int sequenceNumber) {
     					}
     				}
     			}
    +
    +			if (success && backlog > 0) {
    +				onSenderBacklog(backlog);
    +			}
    +
     		} finally {
     			if (!success) {
     				buffer.recycle();
     			}
     		}
     	}
     
    -	public void onEmptyBuffer(int sequenceNumber) {
    +	public void onEmptyBuffer(int sequenceNumber, int backlog) {
    +		boolean success = false;
    +
     		synchronized (receivedBuffers) {
     			if (!isReleased.get()) {
     				if (expectedSequenceNumber == sequenceNumber) {
     					expectedSequenceNumber++;
    +					success = true;
     				} else {
     					onError(new BufferReorderingException(expectedSequenceNumber, sequenceNumber));
     				}
     			}
     		}
    +
    +		if (success && backlog > 0) {
    --- End diff --
    
    same here


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152860980
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -506,36 +540,22 @@ public Void call() throws Exception {
     	@Test
     	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    -		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(248, 32, MemoryType.HEAP);
    +		final int numExclusiveSegments = 120;
    +		final int numFloatingBuffers = 128;
    +		final int backlog = 128;
    +
    +		final ExecutorService executor = Executors.newFixedThreadPool(3);
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
     		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 128;
    -			final int numExclusiveSegments = 2;
     			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
     			inputGate.setBufferPool(bufferPool);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
     
    -			// Exhaust all the floating buffers
    -			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    -			for (int i = 0; i < numFloatingBuffers; i++) {
    -				Buffer buffer = bufferPool.requestBuffer();
    -				assertNotNull(buffer);
    -				floatingBuffers.add(buffer);
    -			}
    -
    -			// Exhaust all the exclusive buffers
    -			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    -			for (int i = 0; i < numExclusiveSegments; i++) {
    -				Buffer buffer = inputChannel.requestBuffer();
    -				assertNotNull(buffer);
    -				exclusiveBuffers.add(buffer);
    -			}
    -
    -			final int backlog = 128;
    -			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +			final Callable requestBufferTask = new Callable<Void>() {
    --- End diff --
    
    please keep `Callable<Void>` (or replace by a lambda)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152858602
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			inputChannel.onSenderBacklog(12);
    +			// Continue increasing the backlog
    +			inputChannel.onSenderBacklog(11);
     
    -			// Already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel is already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle the floating buffer and assign it to the buffer listener
    +			floatingBuffer1.recycle();
    +
    +			// The channel is still waiting for one more floating buffer
    +			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer again
    +			floatingBuffer2.recycle();
    +
    +			// The channel already gets all the required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Decrease the backlog and recycle one exclusive buffer
    +			inputChannel.onSenderBacklog(10);
    +			exclusiveBuffer.recycle();
    +
    +			// Return one floating buffer if the number of available buffers is more than required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    --- End diff --
    
    Good!
    We should definitely have such a case, i.e. `inputChannel.getNumberOfRequiredBuffers() < inputChannel.getNumberOfAvailableBuffers()`, and thereby verify that we do not recycle (potentially) announced buffers. (I almost thought they should be released but there I'm wrong - only after a `recycle()` we may give them back to the pool)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152294000
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(recycleBufferTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			final int numRequiredBuffers = backlog + numExclusiveSegments;
    +			assertEquals("There should be " + numRequiredBuffers +" buffers available in channel.",
    +				numRequiredBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be no buffers available in buffer pool.",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
     		}
    -		inputChannel.assignExclusiveSegments(exclusiveSegments);
    +	}
     
    -		assertEquals("The number of available buffers is not equal to the assigned amount.",
    -			numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * recycling the exclusive or floating buffers and some other thread releasing the
    +	 * input channel.
    +	 */
    +	@Test
    +	public void testConcurrentRecycleAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(recycleBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
    --- End diff --
    
    also verify the `bufferPool.getNumberOfAvailableMemorySegments()` and `networkBufferPool.getNumberOfAvailableMemorySegments`?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141794108
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    --- End diff --
    
    Actually I implemented this strategy in two different ways on our production.
    
    On `LocalBufferPool` side, it has the ability to assign available buffers among all the listeners in round-robin fair way because it can gather all the listeners within some time. But it may bring delay by triggering assignment on `LocalBufferPool` side.
    
    On `RemoteInputChannel` side, we currently implement another complicated way to request buffers in a relatively fair way. That is : 
    
    1. Define a parameter `numBuffersPerAllocation` to indicate how many buffers at most to request from `LocalBufferPool` each time.
    2. `min(numBuffersPerAllocation, backlog)` is the actual value to request from `LocalBufferPool`, so one channel will not occupy all the floating buffers, even though its backlog is really large.
    3. In general `numBuffersPerAllocation` should be larger than 1 to avoid throughput decline. For example, if the floating buffers in `LocalBufferPool` can satisfy all the requirements of `RemoteInputChannel`, it is better to notify the producer batch of credits each time than one credit at a time by many times.
    4. On `LocalBufferPool` side, the `RemoteInputChannel` may still register as listener after already requested `numBuffersPerAllocation` buffers when the number of available buffers plus `numBuffersPerAllocation` is less than `backlog`. Then it has to wait for `LocalBufferPool#recycle()` to trigger distributing the left available buffers among all the listeners.
    
    BTW, I did not understand clearly of the formula you mentioned above `backlog + initialCredit - currentCredit`.  I think the initial credit should not be considered in the following interactions. `backlog-currentCredit` can reflect the number of extra buffers needed in real time for each interaction. I know `backlog-currentCredit` is not very accurate because some credits may be already in-flight notification. But it can be balanced in the long run.
    
    What do you think of this way? 


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141665866
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    --- End diff --
    
    "from *the* buffer pool" 


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141288898
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    --- End diff --
    
    We may also have a sender backlog in this `BufferResponse`, don't we? So (as below) add this as the second parameter to `RemoteInputChannel#onEmptyBuffer()`?
    
    Also, please add a test for this use case (if possible) since it does not seem covered at the moment.


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , I divided the original `testRequestAndReturnFloatingBuffer` into three separate tests 'testAvailableBuffersLessThanRequiredBuffers`, `testAvailableBuffersEqualToRequiredBuffers` and `testAvailableBuffersMoreThanRequiredBuffers`. 
    
    And verifies the following behaviors among them:
    1. Recycle the exclusive buffer
    2. Recycle the floating buffer
    3. Decrease the backlog 


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141377976
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    --- End diff --
    
    agree with it


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152963546
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -330,64 +332,120 @@ public void testRequestAndReturnFloatingBuffer() throws Exception {
     			// Prepare the exclusive and floating buffers to verify recycle logic later
     			Buffer exclusiveBuffer = inputChannel.requestBuffer();
     			assertNotNull(exclusiveBuffer);
    -			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    -			assertNotNull(floatingBuffer1);
    -			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    -			assertNotNull(floatingBuffer2);
    +
    +			final int numRecycleFloatingBuffers = 4;
    +			final ArrayDeque<Buffer> floatingBufferQueue = new ArrayDeque<>(numRecycleFloatingBuffers);
    +			for (int i = 0; i < numRecycleFloatingBuffers; i++) {
    +				Buffer floatingBuffer = bufferPool.requestBuffer();
    +				assertNotNull(floatingBuffer);
    +				floatingBufferQueue.add(floatingBuffer);
    +			}
     
     			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers.
    +			// One exclusive buffer is taken before, so we should request 13 floating buffers.
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel",
    +				10, inputChannel.getNumberOfRequiredBuffers());
     
     			// Increase the backlog to exceed the number of available floating buffers
     			inputChannel.onSenderBacklog(10);
     
     			// The channel does not get enough floating buffer and register as buffer listener
    -			verify(bufferPool, times(13)).requestBuffer();
    +			verify(bufferPool, times(15)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			assertEquals("There should be 11 buffers available in the channel",
    +				11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel",
    +				12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
     
     			// Continue increasing the backlog
    -			inputChannel.onSenderBacklog(11);
    +			inputChannel.onSenderBacklog(12);
     
     			// The channel is already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(13)).requestBuffer();
    +			verify(bufferPool, times(15)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			assertEquals("There should be 11 buffers available in the channel",
    +				11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 14 buffers required in the channel",
    +				14, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			// Recycle the floating buffer and assign it to the buffer listener
    -			floatingBuffer1.recycle();
    +			// Recycle one floating buffer
    +			floatingBufferQueue.poll().recycle();
     
    -			// The channel is still waiting for one more floating buffer
    -			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			// Assign the floating buffer to the listener and the channel is still waiting for more floating buffers
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 14 buffers required in the channel",
    +				14, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			// Recycle one more floating buffer again
    -			floatingBuffer2.recycle();
    +			// Recycle one more floating buffer
    +			floatingBufferQueue.poll().recycle();
     
    -			// The channel already gets all the required buffers
    -			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			// Assign the floating buffer to the listener and the channel is still waiting for more floating buffers
    +			assertEquals("There should be 13 buffers available in the channel",
    +				13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 14 buffers required in the channel",
    +				14, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			// Decrease the backlog and recycle one exclusive buffer
    +			// Decrease the backlog
     			inputChannel.onSenderBacklog(10);
    +
    +			// The number of available buffers is already more than required buffers, but the channel is still
    +			// in the status of waiting for buffers on local pool side
    +			assertEquals("There should be 13 buffers available in the channel",
    +				13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel",
    +				12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer
    +			floatingBufferQueue.poll().recycle();
    +
    +			// Assign the floating buffer to the listener, but the channel will return this buffer immediately to the
    +			// local pool because it has already enough available buffers, and will not waiting for buffers any more
    --- End diff --
    
    `it already has` and `will not wait`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141291264
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java ---
    @@ -122,6 +113,28 @@ public void testReceiveEmptyBuffer() throws Exception {
     	}
     
     	/**
    +	 * Verifies that {@link RemoteInputChannel#onSenderBacklog(int)} is called when a
    +	 * {@link BufferResponse} is received.
    +	 */
    +	@Test
    +	public void testReceiveBacklog() throws Exception {
    +		final RemoteInputChannel inputChannel = mock(RemoteInputChannel.class);
    +		when(inputChannel.getInputChannelId()).thenReturn(new InputChannelID());
    +		when(inputChannel.requestBuffer()).thenReturn(TestBufferFactory.createBuffer());
    +
    +		final int backlog = 10;
    +		final BufferResponse receivedBuffer = createBufferResponse(
    +			TestBufferFactory.createBuffer(), 0, inputChannel.getInputChannelId(), backlog);
    +
    +		final CreditBasedClientHandler client = new CreditBasedClientHandler();
    +		client.addInputChannel(inputChannel);
    +
    +		client.channelRead(mock(ChannelHandlerContext.class), receivedBuffer);
    +
    +		verify(inputChannel, times(1)).onSenderBacklog(backlog);
    --- End diff --
    
    Please also add this check to the `testReceiveEmptyBuffer()` method above.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152856498
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			inputChannel.onSenderBacklog(12);
    +			// Continue increasing the backlog
    +			inputChannel.onSenderBacklog(11);
     
    -			// Already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel is already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle the floating buffer and assign it to the buffer listener
    +			floatingBuffer1.recycle();
    +
    +			// The channel is still waiting for one more floating buffer
    +			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer again
    +			floatingBuffer2.recycle();
    +
    +			// The channel already gets all the required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Decrease the backlog and recycle one exclusive buffer
    --- End diff --
    
    please make this two separate tests, i.e. check the invariants after decreasing the backlog size (nothing should change here yet) and once again after recycling an exclusive buffer (possibly integrated with the tests I requested above)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141890343
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from the producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from the buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    +					Buffer buffer = null;
    +					try {
    +						buffer = inputGate.getBufferPool().requestBuffer();
    +					} catch (IOException ex) {
    +						ExceptionUtils.rethrow(ex);
    +					}
    +
    +					if (buffer != null) {
    +						availableBuffers.add(buffer);
    +						numRequestedBuffers++;
    +						continue;
    --- End diff --
    
    How about using the rest in an `else` statement, instead of `continue`ing here? This would be more intuitive, I guess.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141865630
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    --- End diff --
    
    can you mark this method `@Nullable`?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152239228
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -462,4 +554,42 @@ public String getMessage() {
     				expectedSequenceNumber, actualSequenceNumber);
     		}
     	}
    +
    +	private class AvailableBufferQueue {
    +
    +		/** The current available floating buffers from the fixed buffer pool. */
    +		private final ArrayDeque<Buffer> floatingBuffers;
    +
    +		/** The current available exclusive buffers from the global buffer pool. */
    +		private final ArrayDeque<Buffer> exclusiveBuffers;
    +
    +		AvailableBufferQueue() {
    +			this.exclusiveBuffers = new ArrayDeque<>();
    +			this.floatingBuffers = new ArrayDeque<>();
    +		}
    +
    +		void addExclusiveBuffer(Buffer buffer) {
    +			exclusiveBuffers.add(buffer);
    +		}
    +
    +		Buffer takeExclusiveBuffer() {
    +			return exclusiveBuffers.poll();
    +		}
    +
    +		void addFloatingBuffer(Buffer buffer) {
    +			floatingBuffers.add(buffer);
    +		}
    +
    +		Buffer takeFloatingBuffer() {
    --- End diff --
    
    should be `@Nullable`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r142120202
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java ---
    @@ -131,6 +136,63 @@ public void recycle(MemorySegment segment) {
     		availableMemorySegments.add(segment);
     	}
     
    +	public List<MemorySegment> requestMemorySegments(int numRequiredBuffers) throws IOException {
    +		checkArgument(numRequiredBuffers > 0, "The number of required buffers should be larger than 0.");
    +
    +		synchronized (factoryLock) {
    +			if (isDestroyed) {
    +				throw new IllegalStateException("Network buffer pool has already been destroyed.");
    +			}
    +
    +			if (numTotalRequiredBuffers + numRequiredBuffers > totalNumberOfMemorySegments) {
    +				throw new IOException(String.format("Insufficient number of network buffers: " +
    --- End diff --
    
    yes, I think it can be addressed in a separate JIRA. :)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152454790
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -394,7 +419,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode unless
    +	 * the channel has been released.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	@Nullable
    +	public Buffer requestBuffer() {
    +		synchronized (bufferQueue) {
    +			// Take the floating buffer first if possible.
    +			if (bufferQueue.getFloatingBufferSize() > 0) {
    +				return bufferQueue.takeFloatingBuffer();
    +			} else {
    +				return bufferQueue.takeExclusiveBuffer();
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from the producer's buffer response. If the number of available
    +	 * buffers is less than backlog + initialCredit, it will request floating buffers from the buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	void onSenderBacklog(int backlog) throws IOException {
    --- End diff --
    
    yes


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152860104
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(recycleBufferTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			final int numRequiredBuffers = backlog + numExclusiveSegments;
    +			assertEquals("There should be " + numRequiredBuffers +" buffers available in channel.",
    +				numRequiredBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be no buffers available in buffer pool.",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
     		}
    -		inputChannel.assignExclusiveSegments(exclusiveSegments);
    +	}
     
    -		assertEquals("The number of available buffers is not equal to the assigned amount.",
    -			numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * recycling the exclusive or floating buffers and some other thread releasing the
    +	 * input channel.
    +	 */
    +	@Test
    +	public void testConcurrentRecycleAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    --- End diff --
    
    I was actually hoping we could extract more into a common test method but it's probably best as you implemented it to keep the actual tests easier to understand


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152454439
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -82,17 +84,19 @@
     	/** The initial number of exclusive buffers assigned to this channel. */
     	private int initialCredit;
     
    -	/** The current available buffers including both exclusive buffers and requested floating buffers. */
    -	private final ArrayDeque<Buffer> availableBuffers = new ArrayDeque<>();
    +	/** The available buffer queue wraps both exclusive and requested floating buffers. */
    +	private final AvailableBufferQueue bufferQueue = new AvailableBufferQueue();
     
     	/** The number of available buffers that have not been announced to the producer yet. */
     	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
     
     	/** The number of unsent buffers in the producer's sub partition. */
    -	private final AtomicInteger senderBacklog = new AtomicInteger(0);
    +	@GuardedBy("bufferQueue")
    +	private int senderBacklog;
    --- End diff --
    
    agree with it


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141290289
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java ---
    @@ -193,7 +193,8 @@ private void writeAndFlushNextMessageIfPossible(final Channel channel) throws IO
     						BufferResponse msg = new BufferResponse(
     							next.buffer(),
     							reader.getSequenceNumber(),
    -							reader.getReceiverId());
    +							reader.getReceiverId(),
    +							0);
    --- End diff --
    
    I suspect, this is changed to the actual value by the next PR, isn't it?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152279228
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -394,7 +419,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode unless
    +	 * the channel has been released.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	@Nullable
    +	public Buffer requestBuffer() {
    +		synchronized (bufferQueue) {
    +			// Take the floating buffer first if possible.
    +			if (bufferQueue.getFloatingBufferSize() > 0) {
    +				return bufferQueue.takeFloatingBuffer();
    +			} else {
    +				return bufferQueue.takeExclusiveBuffer();
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from the producer's buffer response. If the number of available
    +	 * buffers is less than backlog + initialCredit, it will request floating buffers from the buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	void onSenderBacklog(int backlog) throws IOException {
    --- End diff --
    
    `@VisibleForTesting`? (otherwise it could be private)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141667517
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    +					Buffer buffer = null;
    +					try {
    +						buffer = inputGate.getBufferPool().requestBuffer();
    +					} catch (IOException ex) {
    --- End diff --
    
    can we simply throw this `IOException` instead of wrapping it?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152242064
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -134,9 +138,9 @@ void assignExclusiveSegments(List<MemorySegment> segments) {
     
    --- End diff --
    
    How about adding (most of) this method's body into the `AvailableBufferQueue` class, including some more data handling logic? This way, we can separate the network handling a bit from the buffer logic. (I'll add comments to some more things to extract below)
    Also, you could test the buffer logic alone (inside `AvailableBufferQueue`) a bit better.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141396863
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    +					return;
    +				}
    +
    +				Buffer buffer = inputChannel.requestBuffer();
    +				if (buffer != null) {
    +					buffer.setSize(bufferOrEvent.getSize());
    +					bufferOrEvent.getNettyBuffer().readBytes(buffer.getNioBuffer());
    +
    +					inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +					inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    +				} else if (inputChannel.isReleased()) {
    +					cancelRequestFor(bufferOrEvent.receiverId);
    +				} else {
    +					throw new IOException("There should always have available buffer for credit-based.");
    +				}
    +			} else {
    +				// ---- Event -------------------------------------------------
    +				// TODO We can just keep the serialized data in the Netty buffer and release it later at the reader
    +				byte[] byteArray = new byte[bufferOrEvent.getSize()];
    +				bufferOrEvent.getNettyBuffer().readBytes(byteArray);
    +
    +				MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray);
    +				Buffer buffer = new Buffer(memSeg, FreeingBufferRecycler.INSTANCE, false);
    +
    +				inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +				inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    --- End diff --
    
    yes, I guess, you can use an invalid backlog (`-1`?) to differentiate? Don't put too much effort into it since `PartitionRequestClientHandler` will be removed at some point.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141396023
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    --- End diff --
    
    I saw, but let's sneak this change in ;)
    (`CreditBasedClientHandler` is a new file and and although I know that it will eventually replace `PartitionRequestClientHandler`, I'd like it to not keep too much legacy stuff)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152859784
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -715,4 +686,58 @@ private RemoteInputChannel createRemoteInputChannel(
     			initialAndMaxRequestBackoff._2(),
     			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
     	}
    +
    +	private Callable recycleExclusiveBufferTask(RemoteInputChannel inputChannel, int numExclusiveSegments) {
    +		final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +		// Exhaust all the exclusive buffers
    +		for (int i = 0; i < numExclusiveSegments; i++) {
    +			Buffer buffer = inputChannel.requestBuffer();
    +			assertNotNull(buffer);
    +			exclusiveBuffers.add(buffer);
    +		}
    +
    +		return new Callable<Void>() {
    +			@Override
    +			public Void call() throws Exception {
    +				for (Buffer buffer : exclusiveBuffers) {
    +					buffer.recycle();
    +				}
    +
    +				return null;
    +			}
    +		};
    +	}
    +
    +	private Callable recycleFloatingBufferTask(BufferPool bufferPool, int numFloatingBuffers) throws Exception {
    +		final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +		// Exhaust all the floating buffers
    +		for (int i = 0; i < numFloatingBuffers; i++) {
    +			Buffer buffer = bufferPool.requestBuffer();
    +			assertNotNull(buffer);
    +			floatingBuffers.add(buffer);
    +		}
    +
    +		return new Callable<Void>() {
    +			@Override
    +			public Void call() throws Exception {
    +				for (Buffer buffer : floatingBuffers) {
    +					buffer.recycle();
    +				}
    +
    +				return null;
    +			}
    +		};
    +	}
    +
    +	private void submitTasksAndWaitResults(ExecutorService executor, Callable[] tasks) throws Exception {
    --- End diff --
    
    please add a Javadoc


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r142142736
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java ---
    @@ -235,14 +240,15 @@ void releaseBuffer() {
     		ByteBuf write(ByteBufAllocator allocator) throws IOException {
     			checkNotNull(buffer, "No buffer instance to serialize.");
     
    -			int length = 16 + 4 + 1 + 4 + buffer.getSize();
    +			int length = 16 + 4 + 4 + 1 + 4 + buffer.getSize();
    --- End diff --
    
    I think the lengths for the corresponding fields can be seen clearly in the below write method.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152961281
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    --- End diff --
    
    Sorry about the forth-and-back here, but thinking about the bug that you fixed with the latest commit, it would be dangerous if we ever called `maintainTargetSize()` without adding an exclusive buffer. What do you think about my second suggestion instead, i.e. having a
    ```
    		/**
    		 * Adds an exclusive buffer (back) into the queue and recycles one floating buffer if the
    		 * number of available buffers in queue is more than the required amount.
    		 *
    		 * @param buffer
    		 * 		the exclusive buffer to add
    		 * @param numRequiredBuffers
    		 * 		the number of required buffers
    		 *
    		 * @return how many buffers were added to the queue
    		 */
    		int addExclusiveBuffer(Buffer buffer, int numRequiredBuffers) {
    			exclusiveBuffers.add(buffer);
    			if (getAvailableBufferSize() > numRequiredBuffers) {
    				Buffer floatingBuffer = floatingBuffers.poll();
    				floatingBuffer.recycle();
    				return 0;
    			} else {
    				return 1;
    			}
    		}
    ```
    (please note the changed return type which I think is more obvious than a `boolean`)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152962580
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -330,64 +332,120 @@ public void testRequestAndReturnFloatingBuffer() throws Exception {
     			// Prepare the exclusive and floating buffers to verify recycle logic later
     			Buffer exclusiveBuffer = inputChannel.requestBuffer();
     			assertNotNull(exclusiveBuffer);
    -			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    -			assertNotNull(floatingBuffer1);
    -			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    -			assertNotNull(floatingBuffer2);
    +
    +			final int numRecycleFloatingBuffers = 4;
    +			final ArrayDeque<Buffer> floatingBufferQueue = new ArrayDeque<>(numRecycleFloatingBuffers);
    +			for (int i = 0; i < numRecycleFloatingBuffers; i++) {
    +				Buffer floatingBuffer = bufferPool.requestBuffer();
    +				assertNotNull(floatingBuffer);
    +				floatingBufferQueue.add(floatingBuffer);
    +			}
    --- End diff --
    
    if you add `verify(bufferPool, times(4)).requestBuffer();` here, the difference to the 13 below becomes a bit more clear (because the requests are absolute values while as differences would be clear from the start, i.e. that we request 9 additional buffers - unfortunately, Mockito does not provide this as far as I know - but that's not a big deal as soon as everything is clear)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141890051
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    +					Buffer buffer = null;
    +					try {
    +						buffer = inputGate.getBufferPool().requestBuffer();
    +					} catch (IOException ex) {
    --- End diff --
    
    This only affects the two methods you mentioned which, themselves, are called by `PartitionRequestClientHandler`/`CreditBasedClientHandler` helper methods allowing `Throwable` - so it's actually not much. I'd prefer not wrapping the exception further here for simplicity in case of errors


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141285605
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    --- End diff --
    
    Since this is actually only calling the `super` method, we can remove our implementation of the method, or am I not seeing something here?.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141665789
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    --- End diff --
    
    "from *the* producer's buffer response"


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r153441043
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,12 +306,10 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    -
    -			bufferQueue.addExclusiveBuffer(new Buffer(segment, this));
    -			floatingBufferRecycled = bufferQueue.maintainTargetSize(numRequiredBuffers);
    +			numAddedBuffers = bufferQueue.addExclusiveBuffer(new Buffer(segment, this), numRequiredBuffers);
     		}
     
    -		if (!floatingBufferRecycled && unannouncedCredit.getAndAdd(1) == 0) {
    +		if (numAddedBuffers > 0 && unannouncedCredit.getAndAdd(1) == 0) {
    --- End diff --
    
    shouldn't this be `if (numAddedBuffers > 0 && unannouncedCredit.getAndAdd(numAddedBuffers) == 0) {` now? (although we only use 0 or 1 as return values for now)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r153253247
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    --- End diff --
    
    Accidentally found that this was not addressed in the newest commits...can you change this?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152972315
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -729,6 +804,12 @@ public Void call() throws Exception {
     		};
     	}
     
    +	/**
    +	 * Submits all the callable tasks to the executor and waits the results.
    --- End diff --
    
    `waits for the results`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141377486
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    --- End diff --
    
    It also keeps the same with current `PartitionRequestClientHandler`, then I can remove it.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141896356
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -412,23 +468,35 @@ public void onBuffer(Buffer buffer, int sequenceNumber) {
     					}
     				}
     			}
    +
    +			if (success && backlog > 0) {
    --- End diff --
    
    Shouldn't we call this for `success && backlog >= 0` to try to always have `initialCredit` (extra) buffers available (if there are enough buffers)?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152252414
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,62 +314,79 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    -			availableBuffers.add(new Buffer(segment, this));
    +
    +			// Recycle one extra floating buffer to always maintain (senderBacklog + initialCredit) buffers available.
    +			if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) {
    --- End diff --
    
    extract this into `AvailableBufferQueue#maintainTargetSize()`?


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK, I have submitted the updates addressed all the comments. Wish your continued reviews! :)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152513656
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    --- End diff --
    
    agree with it


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152972034
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			inputChannel.onSenderBacklog(12);
    +			// Continue increasing the backlog
    +			inputChannel.onSenderBacklog(11);
     
    -			// Already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel is already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle the floating buffer and assign it to the buffer listener
    +			floatingBuffer1.recycle();
    +
    +			// The channel is still waiting for one more floating buffer
    +			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer again
    +			floatingBuffer2.recycle();
    +
    +			// The channel already gets all the required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    --- End diff --
    
    Great. I'm unsure, however, whether you tested all three cases now, i.e. getting from a state of `getNumberOfRequiredBuffers() > getNumberOfAvailableBuffers()` to a state of equality via all three means. Your last tests all come from `getNumberOfRequiredBuffers() <= getNumberOfAvailableBuffers()` only for which you check the behaviour. Also, `testRequestAndReturnFloatingBuffer()` is getting quite long and complex now.
    
    Should we have those as (three?) separate tests instead (and cover these two origins one after another there, starting with `getNumberOfRequiredBuffers() > getNumberOfAvailableBuffers()` first, getting to `getNumberOfRequiredBuffers() = getNumberOfAvailableBuffers()` via one of the three causes, and getting to `getNumberOfRequiredBuffers() <= getNumberOfAvailableBuffers()` via one of them, too)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143425927
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -434,6 +435,52 @@ public void testWaitForFloatingBuffersOnBuffer() throws Exception {
     		verify(bufferPool, times(1)).requestBuffer();
     	}
     
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers and some other thread recycling them.
    +	 */
    +	@Test
    +	public void testConcurrentRequestBufferAndNotifyBufferAvailable() throws Exception {
    +		// Setup
    +		final ExecutorService executor = Executors.newFixedThreadPool(1);
    +		final Buffer buffer = TestBufferFactory.createBuffer();
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(null);
    +		when(bufferPool.addBufferListener(any(BufferListener.class))).thenReturn(true);
    +
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		when(inputGate.getBufferPool()).thenReturn(bufferPool);
    +		when(inputGate.getBufferProvider()).thenReturn(bufferPool);
    +		try {
    +			final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +			// Trigger to request one floating buffer on sender backlog
    +			inputChannel.onBuffer(buffer, 0, 1);
    +
    +			final CountDownLatch sync = new CountDownLatch(1);
    +
    +			// Submit task and wait to finish
    +			Future<Void> result = executor.submit(new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.notifyBufferAvailable(buffer);
    +					sync.countDown();
    +
    +					return null;
    +				}
    +			});
    --- End diff --
    
    This will not run the code in parallel - `onBuffer()` is always executed before `inputChannel.notifyBufferAvailable(buffer);`.
    
    Among the tests I recently stumbled upon, `BlobServerGetTest#testConcurrentGetOperations()` may be a good base to start a new concurrency test with.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152239190
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -462,4 +554,42 @@ public String getMessage() {
     				expectedSequenceNumber, actualSequenceNumber);
     		}
     	}
    +
    +	private class AvailableBufferQueue {
    +
    +		/** The current available floating buffers from the fixed buffer pool. */
    +		private final ArrayDeque<Buffer> floatingBuffers;
    +
    +		/** The current available exclusive buffers from the global buffer pool. */
    +		private final ArrayDeque<Buffer> exclusiveBuffers;
    +
    +		AvailableBufferQueue() {
    +			this.exclusiveBuffers = new ArrayDeque<>();
    +			this.floatingBuffers = new ArrayDeque<>();
    +		}
    +
    +		void addExclusiveBuffer(Buffer buffer) {
    +			exclusiveBuffers.add(buffer);
    +		}
    +
    +		Buffer takeExclusiveBuffer() {
    --- End diff --
    
    should be `@Nullable`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152457595
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -240,15 +245,13 @@ void releaseAllResources() throws IOException {
     					}
     				}
     			}
    -
    -			synchronized (availableBuffers) {
    +			synchronized (bufferQueue) {
    --- End diff --
    
    ok


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152239410
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -462,4 +554,42 @@ public String getMessage() {
     				expectedSequenceNumber, actualSequenceNumber);
     		}
     	}
    +
    +	private class AvailableBufferQueue {
    --- End diff --
    
    Please add some (small) description to the Javadoc of this class.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152859775
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -715,4 +686,58 @@ private RemoteInputChannel createRemoteInputChannel(
     			initialAndMaxRequestBackoff._2(),
     			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
     	}
    +
    +	private Callable recycleExclusiveBufferTask(RemoteInputChannel inputChannel, int numExclusiveSegments) {
    +		final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +		// Exhaust all the exclusive buffers
    +		for (int i = 0; i < numExclusiveSegments; i++) {
    +			Buffer buffer = inputChannel.requestBuffer();
    +			assertNotNull(buffer);
    +			exclusiveBuffers.add(buffer);
    +		}
    +
    +		return new Callable<Void>() {
    +			@Override
    +			public Void call() throws Exception {
    +				for (Buffer buffer : exclusiveBuffers) {
    +					buffer.recycle();
    +				}
    +
    +				return null;
    +			}
    +		};
    +	}
    +
    +	private Callable recycleFloatingBufferTask(BufferPool bufferPool, int numFloatingBuffers) throws Exception {
    +		final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    --- End diff --
    
    please add a Javadoc


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152899386
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			inputChannel.onSenderBacklog(12);
    +			// Continue increasing the backlog
    +			inputChannel.onSenderBacklog(11);
     
    -			// Already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel is already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle the floating buffer and assign it to the buffer listener
    +			floatingBuffer1.recycle();
    +
    +			// The channel is still waiting for one more floating buffer
    +			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer again
    +			floatingBuffer2.recycle();
    +
    +			// The channel already gets all the required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    --- End diff --
    
    These new additional checks help me find an incorrect point in `RemoteInputChannel#notifyBufferAvailable`. Thanks for the suggestions! 


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141288399
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    +					return;
    +				}
    +
    +				Buffer buffer = inputChannel.requestBuffer();
    +				if (buffer != null) {
    +					buffer.setSize(bufferOrEvent.getSize());
    +					bufferOrEvent.getNettyBuffer().readBytes(buffer.getNioBuffer());
    +
    +					inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +					inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    +				} else if (inputChannel.isReleased()) {
    +					cancelRequestFor(bufferOrEvent.receiverId);
    +				} else {
    +					throw new IOException("There should always have available buffer for credit-based.");
    +				}
    +			} else {
    +				// ---- Event -------------------------------------------------
    +				// TODO We can just keep the serialized data in the Netty buffer and release it later at the reader
    +				byte[] byteArray = new byte[bufferOrEvent.getSize()];
    +				bufferOrEvent.getNettyBuffer().readBytes(byteArray);
    +
    +				MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray);
    +				Buffer buffer = new Buffer(memSeg, FreeingBufferRecycler.INSTANCE, false);
    +
    +				inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +				inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    --- End diff --
    
    It seems that these two calls always belong together. Maybe add `bufferOrEvent.backlog` as the third parameter to `RemoteInputChannel#onBuffer()`?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141377249
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    --- End diff --
    
    It keeps the same with current `PartitionRequestClientHandler`, then I can modify that.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143233556
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -331,37 +353,41 @@ public int getNumberOfAvailableBuffers() {
     	 */
     	@Override
     	public boolean notifyBufferAvailable(Buffer buffer) {
    -		checkState(isWaitingForFloatingBuffers.get(), "This channel should be waiting for floating buffers.");
    +		checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers.");
     
    +		boolean needMoreBuffers;
     		synchronized (availableBuffers) {
     			// Important: the isReleased check should be inside the synchronized block.
    -			if (isReleased.get() || availableBuffers.size() >= senderBacklog.get()) {
    -				isWaitingForFloatingBuffers.set(false);
    +			if (isReleased.get() || availableBuffers.size() >= senderBacklog.get() + initialCredit) {
    --- End diff --
    
    1) The `senderBacklog` is only ever accessed under the `synchronized(availableBuffers)`. I can't really think of a reason to drag it out of there and keep it an `AtomicInteger` at the moment - what do you think about making it a simple `int` instead?
    2) Since we release extraneous floating buffers in `recycle()`, I guess, `availableBuffers.size() > senderBacklog.get() + initialCredit` is not possible here, is it? Should we `checkState()` this and otherwise ignore this case in here?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143680178
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,10 +311,27 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    +
    +			// Recycle the extra floating buffers in order not to stack up 2*initialCredit
    +			// buffers once current backlog is 0
    +			if (senderBacklog.get() == 0 && availableBuffers.size() >= initialCredit) {
    --- End diff --
    
    Yes, I understand your point. 
    I only considered recycling extra floating buffers when no backlog in sender before, actually it is more reasonable to check and recycle extra floating buffer after every exclusive buffer returned.


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , thanks for your reviews on Thanksgiving day. 
    I have updated the codes addressed your comments. : )


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152893996
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    --- End diff --
    
    good point


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152859069
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    --- End diff --
    
    Thanks, the extensions you implemented for `testRequestFloatingBufferOnSenderBacklog()` were exactly what I missed previously and was not covered by `testFairDistributionFloatingBuffers()`. Based on your additions, I came up with some more (corner) cases that should be covered (see above)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by bowenli86 <gi...@git.apache.org>.
Github user bowenli86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141945644
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java ---
    @@ -131,6 +136,63 @@ public void recycle(MemorySegment segment) {
     		availableMemorySegments.add(segment);
     	}
     
    +	public List<MemorySegment> requestMemorySegments(int numRequiredBuffers) throws IOException {
    +		checkArgument(numRequiredBuffers > 0, "The number of required buffers should be larger than 0.");
    +
    +		synchronized (factoryLock) {
    +			if (isDestroyed) {
    +				throw new IllegalStateException("Network buffer pool has already been destroyed.");
    +			}
    +
    +			if (numTotalRequiredBuffers + numRequiredBuffers > totalNumberOfMemorySegments) {
    +				throw new IOException(String.format("Insufficient number of network buffers: " +
    --- End diff --
    
    How about adding a `FlinkNetworkException` or `FlinkNetworkIOException` for all network/netty related exceptions? That can help users better identify root causes of problems.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141396295
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    --- End diff --
    
    same here - I think, I have even removed it in one of my work-in-progress branches


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152291537
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    --- End diff --
    
    why not have two recycling threads, one for the exclusive buffers and another for the floating buffers? (you should then maybe increase the number of exclusive buffers)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152459473
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -134,9 +138,9 @@ void assignExclusiveSegments(List<MemorySegment> segments) {
     
    --- End diff --
    
    agree


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152861029
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -440,21 +476,25 @@ public void testFairDistributionFloatingBuffers() throws Exception {
     	@Test
     	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(130, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 128;
    +
     		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
     		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
     			inputGate.setBufferPool(bufferPool);
    -			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +			final Callable requestBufferTask = new Callable<Void>() {
    --- End diff --
    
    please keep `Callable<Void>` (or replace by a lambda)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152245491
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -240,15 +245,13 @@ void releaseAllResources() throws IOException {
     					}
     				}
     			}
    -
    -			synchronized (availableBuffers) {
    +			synchronized (bufferQueue) {
    --- End diff --
    
    add a `AvailableBufferQueue#releaseAll(exclusiveRecyclingSegments)`?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152836741
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    +			exclusiveBuffers.add(buffer);
    +
    +			if (getAvailableBufferSize() > numRequiredBuffers) {
    +				Buffer floatingBuffer = floatingBuffers.poll();
    +				floatingBuffer.recycle();
    +				return true;
    +			} else {
    +				return false;
    +			}
     		}
     
    -		int getFloatingBufferSize() {
    -			return floatingBuffers.size();
    +		/**
    +		 * Take the floating buffer first if possible.
    +		 */
    +		@Nullable
    +		Buffer takeBuffer() {
    +			if (floatingBuffers.size() > 0) {
    +				return floatingBuffers.poll();
    +			} else {
    +				return exclusiveBuffers.poll();
    +			}
    +		}
    +
    +		/**
    +		 * The floating buffer is recycled to local buffer pool directly, and the
    +		 * exclusive buffer will be gathered to return to global buffer pool later.
    +		 */
    +		void releaseAll(List<MemorySegment> exclusiveSegments) {
    --- End diff --
    
    please document the `exclusiveSegments` parameter to make it absolutely clear for the user that we will add memory segments of exclusive buffers


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141380216
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java ---
    @@ -193,7 +193,8 @@ private void writeAndFlushNextMessageIfPossible(final Channel channel) throws IO
     						BufferResponse msg = new BufferResponse(
     							next.buffer(),
     							reader.getSequenceNumber(),
    -							reader.getReceiverId());
    +							reader.getReceiverId(),
    +							0);
    --- End diff --
    
    yes


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK  @pnowojski , this PR is ready now. I rebased the latest master code and updated the modifications based on previous comments.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152860672
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(recycleBufferTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			final int numRequiredBuffers = backlog + numExclusiveSegments;
    +			assertEquals("There should be " + numRequiredBuffers +" buffers available in channel.",
    +				numRequiredBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be no buffers available in buffer pool.",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
     		}
    -		inputChannel.assignExclusiveSegments(exclusiveSegments);
    +	}
     
    -		assertEquals("The number of available buffers is not equal to the assigned amount.",
    -			numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * recycling the exclusive or floating buffers and some other thread releasing the
    +	 * input channel.
    +	 */
    +	@Test
    +	public void testConcurrentRecycleAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(recycleBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
    --- End diff --
    
    Seems that you changed your mind here, and I believe that this is correct: after everything has been recycled and the channel is released, everything should be back in the pool, irrespective of the order of execution - in `testConcurrentRecycleAndRelease()`.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141908388
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -374,6 +376,64 @@ public void testReleaseExclusiveBuffers() throws Exception {
     		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
     	}
     
    +	/**
    +	 * Tests {@link BufferPool#requestBuffer()}, verifying the remote input channel tries to request
    +	 * floating buffers once receiving the producer's backlog.
    +	 */
    +	@Test
    +	public void testRequestFloatingBuffersOnBuffer() throws Exception {
    +		// Setup
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(TestBufferFactory.createBuffer());
    +
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		when(inputGate.getBufferPool()).thenReturn(bufferPool);
    +
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +
    +		// Receive the producer's backlog
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 0, 10);
    +		// Need to request 10 floating buffers from buffer pool
    +		verify(bufferPool, times(10)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 1, 8);
    +		// No need to request extra floating buffers from pool because
    +		// there are already 10 available buffers in queue now
    +		verify(bufferPool, times(10)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 2, 11);
    +		// Need to request another floating buffer from pool
    +		verify(bufferPool, times(11)).requestBuffer();
    +	}
    +
    +	/**
    +	 * Tests {@link BufferPool#requestBuffer()}, verifying the remote input channel tries to request
    +	 * floating buffers once receiving the producer's backlog. And it requests from pool only once
    +	 * and registers as listener if there are currently no available buffers in the pool.
    +	 */
    +	@Test
    +	public void testWaitForFloatingBuffersOnBuffer() throws Exception {
    +		// Setup
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(null);
    +		when(bufferPool.addBufferListener(any(BufferListener.class))).thenReturn(true);
    +
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		when(inputGate.getBufferPool()).thenReturn(bufferPool);
    +		when(inputGate.getBufferProvider()).thenReturn(bufferPool);
    +
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +
    +		// Receive the producer's backlog
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 0, 10);
    +		// Request from pool only once if there are no available floating buffers
    +		verify(bufferPool, times(1)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 1, 10);
    +		// Already registers as listener to wait for notifications and will not request any more
    +		verify(bufferPool, times(1)).requestBuffer();
    +	}
    +
    --- End diff --
    
    actually, some more tests would be nice:
    
    - ensuring a fair distribution of buffers to `BufferListener`s
    - to verify that there is no race condition with two things running in parallel: `onSenderBacklog()` or `notifyBufferAvailable()` requesting buffers and some other thread recycling them (floating and/or exclusive ones).


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143230905
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,10 +311,27 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    +
    +			// Recycle the extra floating buffers in order not to stack up 2*initialCredit
    +			// buffers once current backlog is 0
    +			if (senderBacklog.get() == 0 && availableBuffers.size() >= initialCredit) {
    +				final int size = availableBuffers.size();
    +				for (int i = 0; i < size; i++) {
    +					final Buffer buffer = availableBuffers.poll();
    --- End diff --
    
    I wonder if it would make things simpler if we had two separate lists of buffers: one for exclusive buffers and one for floating buffers. Then you would at least not have to iterate through the list. When retrieving buffers, you'd always take the floating buffers first (so we don't get into this situation too often). If that makes it simpler, we could also extract this logic into a separate `AvailableBuffersQueue` class or so. Regarding testability, this may also make sense.


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    Already submit the codes for the missed comments, @NicoK 


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152894450
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			inputChannel.onSenderBacklog(12);
    +			// Continue increasing the backlog
    +			inputChannel.onSenderBacklog(11);
     
    -			// Already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel is already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle the floating buffer and assign it to the buffer listener
    +			floatingBuffer1.recycle();
    +
    +			// The channel is still waiting for one more floating buffer
    +			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer again
    +			floatingBuffer2.recycle();
    +
    +			// The channel already gets all the required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Decrease the backlog and recycle one exclusive buffer
    +			inputChannel.onSenderBacklog(10);
    +			exclusiveBuffer.recycle();
    +
    +			// Return one floating buffer if the number of available buffers is more than required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    --- End diff --
    
    Yes, the already announced credit can not be taken back from the producer. We can only replace one floating buffer to back to pool by recycling one exclusive buffer. :)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141378307
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    +					return;
    +				}
    +
    +				Buffer buffer = inputChannel.requestBuffer();
    +				if (buffer != null) {
    +					buffer.setSize(bufferOrEvent.getSize());
    +					bufferOrEvent.getNettyBuffer().readBytes(buffer.getNioBuffer());
    +
    +					inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +					inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    +				} else if (inputChannel.isReleased()) {
    +					cancelRequestFor(bufferOrEvent.receiverId);
    +				} else {
    +					throw new IOException("There should always have available buffer for credit-based.");
    --- End diff --
    
    Yes, I will consider IllegalStateException or ProtocolException


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r154286901
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,12 +306,10 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    -
    -			bufferQueue.addExclusiveBuffer(new Buffer(segment, this));
    -			floatingBufferRecycled = bufferQueue.maintainTargetSize(numRequiredBuffers);
    +			numAddedBuffers = bufferQueue.addExclusiveBuffer(new Buffer(segment, this), numRequiredBuffers);
     		}
     
    -		if (!floatingBufferRecycled && unannouncedCredit.getAndAdd(1) == 0) {
    +		if (numAddedBuffers > 0 && unannouncedCredit.getAndAdd(1) == 0) {
    --- End diff --
    
    Yes, you are right. I forgot to update the latter one. :)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152972414
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -715,4 +686,58 @@ private RemoteInputChannel createRemoteInputChannel(
     			initialAndMaxRequestBackoff._2(),
     			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
     	}
    +
    +	private Callable recycleExclusiveBufferTask(RemoteInputChannel inputChannel, int numExclusiveSegments) {
    +		final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +		// Exhaust all the exclusive buffers
    +		for (int i = 0; i < numExclusiveSegments; i++) {
    +			Buffer buffer = inputChannel.requestBuffer();
    +			assertNotNull(buffer);
    +			exclusiveBuffers.add(buffer);
    +		}
    +
    +		return new Callable<Void>() {
    +			@Override
    +			public Void call() throws Exception {
    +				for (Buffer buffer : exclusiveBuffers) {
    +					buffer.recycle();
    +				}
    +
    +				return null;
    +			}
    +		};
    +	}
    +
    +	private Callable recycleFloatingBufferTask(BufferPool bufferPool, int numFloatingBuffers) throws Exception {
    +		final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +		// Exhaust all the floating buffers
    +		for (int i = 0; i < numFloatingBuffers; i++) {
    +			Buffer buffer = bufferPool.requestBuffer();
    +			assertNotNull(buffer);
    +			floatingBuffers.add(buffer);
    +		}
    +
    +		return new Callable<Void>() {
    +			@Override
    +			public Void call() throws Exception {
    +				for (Buffer buffer : floatingBuffers) {
    +					buffer.recycle();
    +				}
    +
    +				return null;
    +			}
    +		};
    +	}
    +
    +	private void submitTasksAndWaitResults(ExecutorService executor, Callable[] tasks) throws Exception {
    --- End diff --
    
    maybe also rename to `submitTasksAndWaitForResults`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152292548
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(recycleBufferTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			final int numRequiredBuffers = backlog + numExclusiveSegments;
    +			assertEquals("There should be " + numRequiredBuffers +" buffers available in channel.",
    +				numRequiredBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be no buffers available in buffer pool.",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
     		}
    -		inputChannel.assignExclusiveSegments(exclusiveSegments);
    +	}
     
    -		assertEquals("The number of available buffers is not equal to the assigned amount.",
    -			numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * recycling the exclusive or floating buffers and some other thread releasing the
    +	 * input channel.
    +	 */
    +	@Test
    +	public void testConcurrentRecycleAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    --- End diff --
    
    same here - two recycle threads?
    Also, this method shares a lot of code with the previous test - can you extract the common parts into a shared method? (with only the non-recycling task being different and the checks after the test is run)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r153378748
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    --- End diff --
    
    Sorry I did not see it before, I will update for that.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141380052
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    +					return;
    +				}
    +
    +				Buffer buffer = inputChannel.requestBuffer();
    +				if (buffer != null) {
    +					buffer.setSize(bufferOrEvent.getSize());
    +					bufferOrEvent.getNettyBuffer().readBytes(buffer.getNioBuffer());
    +
    +					inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +					inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    +				} else if (inputChannel.isReleased()) {
    +					cancelRequestFor(bufferOrEvent.receiverId);
    +				} else {
    +					throw new IOException("There should always have available buffer for credit-based.");
    +				}
    +			} else {
    +				// ---- Event -------------------------------------------------
    +				// TODO We can just keep the serialized data in the Netty buffer and release it later at the reader
    +				byte[] byteArray = new byte[bufferOrEvent.getSize()];
    +				bufferOrEvent.getNettyBuffer().readBytes(byteArray);
    +
    +				MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray);
    +				Buffer buffer = new Buffer(memSeg, FreeingBufferRecycler.INSTANCE, false);
    +
    +				inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +				inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    --- End diff --
    
    Yes, I can modify for that, but also need to modify it in `PartitionRequestClientHandler`.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152512614
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    --- End diff --
    
    Actually I tried to test the two logics in two separate tests `testRequestFloatingBufferOnSenderBacklog` and `testFairDistributionFloatingBuffers`.
    
    For `testRequestFloatingBufferOnSenderBacklog`, it only wants to verify the request logic on input channel side.  The key point is that the input channel will not request repeated if it is already as listener in pool.
    
    For 'testFairDistributionFloatingBuffers`, it only wants to verify that the input channel listener is getting buffer fairly during buffer `recycle()` on `bufferPool` side. I think it can cover the comment you mentioned "Can you also verify the behaviour when the buffers become available?" I will further check that later.
    
    I agree with that we missed the tests to verify that we stick to `senderBacklog + initialCredit` in different processes. And I will add them later.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141293477
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java ---
    @@ -169,95 +182,19 @@ public void testCancelBeforeActive() throws Exception {
     		client.cancelRequestFor(inputChannel.getInputChannelId());
     	}
     
    -	/**
    -	 * Tests that an unsuccessful message decode call for a staged message
    -	 * does not leave the channel with auto read set to false.
    -	 */
    -	@Test
    -	@SuppressWarnings("unchecked")
    -	public void testAutoReadAfterUnsuccessfulStagedMessage() throws Exception {
    --- End diff --
    
    should we instead have a simple test that verifies that the `CreditBasedClientHandler` throws in case it has no buffers available? (which is a violation of the protocol but may happen for whatever reason)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141283335
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    --- End diff --
    
    I guess, we can also use `ConcurrentHashMap` here and avoid the guava use


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , thanks for your reviews!
    Actually there are still two PRs under submitted before this PR. One is for floating buffer interaction between `RemoteInputChannel` and `BufferPool`, the other is `onSenderBacklog` logic in `RemoteInputChannel`. 
    I will submit both of them this week and also rebase this PR based on your comments.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141865540
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    --- End diff --
    
    ...unless the channel has been released


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141906648
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -374,6 +376,64 @@ public void testReleaseExclusiveBuffers() throws Exception {
     		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
     	}
     
    +	/**
    +	 * Tests {@link BufferPool#requestBuffer()}, verifying the remote input channel tries to request
    +	 * floating buffers once receiving the producer's backlog.
    +	 */
    +	@Test
    +	public void testRequestFloatingBuffersOnBuffer() throws Exception {
    +		// Setup
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(TestBufferFactory.createBuffer());
    +
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		when(inputGate.getBufferPool()).thenReturn(bufferPool);
    +
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +
    +		// Receive the producer's backlog
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 0, 10);
    +		// Need to request 10 floating buffers from buffer pool
    +		verify(bufferPool, times(10)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 1, 8);
    +		// No need to request extra floating buffers from pool because
    +		// there are already 10 available buffers in queue now
    +		verify(bufferPool, times(10)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 2, 11);
    +		// Need to request another floating buffer from pool
    +		verify(bufferPool, times(11)).requestBuffer();
    +	}
    +
    +	/**
    +	 * Tests {@link BufferPool#requestBuffer()}, verifying the remote input channel tries to request
    +	 * floating buffers once receiving the producer's backlog. And it requests from pool only once
    +	 * and registers as listener if there are currently no available buffers in the pool.
    +	 */
    +	@Test
    +	public void testWaitForFloatingBuffersOnBuffer() throws Exception {
    +		// Setup
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(null);
    +		when(bufferPool.addBufferListener(any(BufferListener.class))).thenReturn(true);
    --- End diff --
    
    How about using the real `NetworkBufferPool#createBufferPool()` here with a limited set of buffers? Then you could start retrieving buffers as in the test method above and continue with verifying the expected behaviour in case the buffer limit was reached (no need for two test methods, I guess). I'd prefer this over a mock so that you can also verify the interaction with the real methods such as `addBufferListener()`.


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , thanks for reviews and talking about this key point.
    
    I think I understand your point and agree with that. I will submit the modifications before Monday.


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , I have submitted the modifications based on your comments.
    
    I will add the description for `CreditBasedClientHandler` module as you suggested in next PR, because the description should also cover the function of writing and flushing credit message.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141673902
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    --- End diff --
    
    By taking all buffers until none are available any more, we do not really ensure a fair distribution where large backlogs on one channel may actually "starve" another (except for the exclusive buffers). Floating buffers should be assigned round-robin among the channels.
    
    Also, actually, the total number of required buffers should be `backlog + initialCredit - currentCredit` with `currentCredit` being the credit that was guaranteed, i.e. `availableBuffers.size()`.
    
    Let me think about this a bit more to sort out the details - I'll come back to you.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152859769
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -715,4 +686,58 @@ private RemoteInputChannel createRemoteInputChannel(
     			initialAndMaxRequestBackoff._2(),
     			new UnregisteredTaskMetricsGroup.DummyTaskIOMetricGroup());
     	}
    +
    +	private Callable recycleExclusiveBufferTask(RemoteInputChannel inputChannel, int numExclusiveSegments) {
    --- End diff --
    
    please add a Javadoc


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , I have fixed the other code issues. 
    
    For UT, I only added one test for verifying  there is no race condition between requesting and recycling floating buffers. I am not sure whether to control the process by `CountDownLatch` is enough. Or I should submit two different threads to execute the process repeated. If this way is ok, I will add more tests for race condition along with `NetworkBufferPool#createBufferPool()` modifications you suggested. 
    
    For fair distribution of buffers test, I will add one test, maybe in this PR or in #4735 later.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152861040
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -465,7 +505,7 @@ public Void call() throws Exception {
     				}
     			};
     
    -			final Callable<Void> releaseTask = new Callable<Void>() {
    +			final Callable releaseTask = new Callable<Void>() {
    --- End diff --
    
    please keep `Callable<Void>` (or replace by a lambda)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152737920
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to global pool via input gate when channel is released.
    +	 * Tests to verify that the buffer pool will distribute available floating buffers among
    +	 * all the channel listeners in a fair way.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferAfterReleased() throws Exception {
    +	public void testFairDistributionFloatingBuffers() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel channel1 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel2 = spy(createRemoteInputChannel(inputGate));
    +		final RemoteInputChannel channel3 = spy(createRemoteInputChannel(inputGate));
    +		try {
    +			final int numFloatingBuffers = 3;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channels
    +			inputGate.setInputChannel(channel1.partitionId.getPartitionId(), channel1);
    +			inputGate.setInputChannel(channel2.partitionId.getPartitionId(), channel2);
    +			inputGate.setInputChannel(channel3.partitionId.getPartitionId(), channel3);
    +			final int numExclusiveBuffers = 2;
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Receive the producer's backlog to trigger request floating buffers from pool
    +			// and register as listeners as a result
    +			channel1.onSenderBacklog(8);
    +			channel2.onSenderBacklog(8);
    +			channel3.onSenderBacklog(8);
    +
    +			verify(bufferPool, times(1)).addBufferListener(channel1);
    +			verify(bufferPool, times(1)).addBufferListener(channel2);
    +			verify(bufferPool, times(1)).addBufferListener(channel3);
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, channel3.getNumberOfAvailableBuffers());
    +
    +			// Recycle three floating buffers to trigger notify buffer available
    +			for (Buffer buffer : floatingBuffers) {
    +				buffer.recycle();
    +			}
    +
    +			verify(channel1, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel2, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			verify(channel3, times(1)).notifyBufferAvailable(any(Buffer.class));
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel1.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel2.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 3 buffers available in the channel", 3, channel3.getNumberOfAvailableBuffers());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			channel1.releaseAllResources();
    +			channel2.releaseAllResources();
    +			channel3.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +		}
    +	}
    +
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread releasing
    +	 * the input channel.
    +	 */
    +	@Test
    +	public void testConcurrentOnSenderBacklogAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(128, 128);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, 2);
    +
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					while (true) {
    +						for (int j = 1; j <= 128; j++) {
    +							inputChannel.onSenderBacklog(j);
    +						}
    +
    +						if (inputChannel.isReleased()) {
    +							return null;
    +						}
    +					}
    +				}
    +			};
     
    -		inputChannel.releaseAllResources();
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
     
    -		// Recycle exclusive segment after channel released
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
     
    -		assertEquals("Resource leak during recycling buffer after channel is released.",
    -			0, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(0)).notifyCreditAvailable();
    -		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
    +		} finally {
    +			// Release all the buffer resources once exception
    +			if (!inputChannel.isReleased()) {
    +				inputChannel.releaseAllResources();
    +			}
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
    +		}
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#releaseAllResources()}, verifying the exclusive segments are
    -	 * recycled to global pool via input gate and no resource leak.
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * requesting floating buffers on sender backlog and some other thread recycling
    +	 * floating or exclusive buffers.
     	 */
     	@Test
    -	public void testReleaseExclusiveBuffers() throws Exception {
    +	public void testConcurrentOnSenderBacklogAndRecycle() throws Exception {
     		// Setup
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
     
    -		// Assign exclusive segments to channel
    -		final List<MemorySegment> exclusiveSegments = new ArrayList<>();
    -		final int numExclusiveBuffers = 2;
    -		for (int i = 0; i < numExclusiveBuffers; i++) {
    -			exclusiveSegments.add(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final int backlog = 128;
    +			final Callable<Void> requestBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					for (int j = 1; j <= backlog; j++) {
    +						inputChannel.onSenderBacklog(j);
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(requestBufferTask));
    +			results.add(executor.submit(recycleBufferTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			final int numRequiredBuffers = backlog + numExclusiveSegments;
    +			assertEquals("There should be " + numRequiredBuffers +" buffers available in channel.",
    +				numRequiredBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be no buffers available in buffer pool.",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +		} finally {
    +			// Release all the buffer resources
    +			inputChannel.releaseAllResources();
    +
    +			networkBufferPool.destroyAllBufferPools();
    +			networkBufferPool.destroy();
    +
    +			executor.shutdown();
     		}
    -		inputChannel.assignExclusiveSegments(exclusiveSegments);
    +	}
     
    -		assertEquals("The number of available buffers is not equal to the assigned amount.",
    -			numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +	/**
    +	 * Tests to verify that there is no race condition with two things running in parallel:
    +	 * recycling the exclusive or floating buffers and some other thread releasing the
    +	 * input channel.
    +	 */
    +	@Test
    +	public void testConcurrentRecycleAndRelease() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(256, 32, MemoryType.HEAP);
    +		final ExecutorService executor = Executors.newFixedThreadPool(2);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel  = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +		try {
    +			final int numFloatingBuffers = 128;
    +			final int numExclusiveSegments = 2;
    +			final BufferPool bufferPool = networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers);
    +			inputGate.setBufferPool(bufferPool);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveSegments);
    +
    +			// Exhaust all the floating buffers
    +			final List<Buffer> floatingBuffers = new ArrayList<>(numFloatingBuffers);
    +			for (int i = 0; i < numFloatingBuffers; i++) {
    +				Buffer buffer = bufferPool.requestBuffer();
    +				assertNotNull(buffer);
    +				floatingBuffers.add(buffer);
    +			}
    +
    +			// Exhaust all the exclusive buffers
    +			final List<Buffer> exclusiveBuffers = new ArrayList<>(numExclusiveSegments);
    +			for (int i = 0; i < numExclusiveSegments; i++) {
    +				Buffer buffer = inputChannel.requestBuffer();
    +				assertNotNull(buffer);
    +				exclusiveBuffers.add(buffer);
    +			}
    +
    +			final Callable<Void> recycleBufferTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					// Recycle all the exclusive buffers
    +					for (Buffer buffer : exclusiveBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					// Recycle all the floating buffers
    +					for (Buffer buffer : floatingBuffers) {
    +						buffer.recycle();
    +					}
    +
    +					return null;
    +				}
    +			};
    +
    +			final Callable<Void> releaseTask = new Callable<Void>() {
    +				@Override
    +				public Void call() throws Exception {
    +					inputChannel.releaseAllResources();
    +
    +					return null;
    +				}
    +			};
    +
    +			// Submit tasks and wait to finish
    +			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    +			results.add(executor.submit(recycleBufferTask));
    +			results.add(executor.submit(releaseTask));
    +			for (Future<Void> result : results) {
    +				result.get();
    +			}
    +
    +			assertEquals("There should be no buffers available in the channel.",
    +				0, inputChannel.getNumberOfAvailableBuffers());
    --- End diff --
    
    When the channel is released by one thread, we are not sure whether the floating buffer is requested and how many are requested by another thread.
    
    If the number of requested floating buffers before is less than the capacity of pool, it can not verify how many available floating buffers currently in the pool after channel released, and the same as `NetworkBufferPool`.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141281604
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    --- End diff --
    
    I know, the original `PartitionRequestClientHandler` has not been documented, but could you please add a small description so that we gradually improve the code style / documentation in this module?


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    merged in 268867ce620a2c12879749db2ecb68bbe129cad5
    @zhijiangW can you close this PR?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152857580
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			inputChannel.onSenderBacklog(12);
    +			// Continue increasing the backlog
    +			inputChannel.onSenderBacklog(11);
     
    -			// Already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel is already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle the floating buffer and assign it to the buffer listener
    +			floatingBuffer1.recycle();
    +
    +			// The channel is still waiting for one more floating buffer
    +			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    +			// Recycle one more floating buffer again
    +			floatingBuffer2.recycle();
    +
    +			// The channel already gets all the required buffers
    +			assertEquals("There should be 13 buffers available in the channel", 13, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +
    --- End diff --
    
    Whenever `inputChannel.getNumberOfAvailableBuffers()` reaches `inputChannel.getNumberOfRequiredBuffers()`, we should not listen to / add new buffers. We should verify this behaviour and it could result from three things:
    1. enough floating buffers become available
    2. enough exclusive buffers become available
    3. the sender backlog decreases
    
    We can verify that we do not listen to new buffers anymore indirectly by recycling yet another floating buffer and verifying that `inputChannel.getNumberOfAvailableBuffers()` does not change.
    
    Can you add these two additional checks?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143232200
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -331,37 +353,41 @@ public int getNumberOfAvailableBuffers() {
     	 */
     	@Override
     	public boolean notifyBufferAvailable(Buffer buffer) {
    -		checkState(isWaitingForFloatingBuffers.get(), "This channel should be waiting for floating buffers.");
    +		checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers.");
     
    +		boolean needMoreBuffers;
     		synchronized (availableBuffers) {
     			// Important: the isReleased check should be inside the synchronized block.
    -			if (isReleased.get() || availableBuffers.size() >= senderBacklog.get()) {
    -				isWaitingForFloatingBuffers.set(false);
    +			if (isReleased.get() || availableBuffers.size() >= senderBacklog.get() + initialCredit) {
    +				isWaitingForFloatingBuffers = false;
     				buffer.recycle();
     
     				return false;
     			}
     
     			availableBuffers.add(buffer);
     
    -			if (unannouncedCredit.getAndAdd(1) == 0) {
    -				notifyCreditAvailable();
    -			}
    -
    -			if (availableBuffers.size() >= senderBacklog.get()) {
    -				isWaitingForFloatingBuffers.set(false);
    -				return false;
    +			if (availableBuffers.size() >= senderBacklog.get() + initialCredit) {
    +				isWaitingForFloatingBuffers = false;
    +				needMoreBuffers = false;
     			} else {
    -				return true;
    +				needMoreBuffers =  true;
     			}
     		}
    +
    +		if (unannouncedCredit.getAndAdd(1) == 0) {
    +			notifyCreditAvailable();
    +		}
    +
    +		return needMoreBuffers;
     	}
     
     	@Override
     	public void notifyBufferDestroyed() {
    -		if (!isWaitingForFloatingBuffers.compareAndSet(true, false)) {
    -			throw new IllegalStateException("This channel should be waiting for floating buffers currently.");
    +		if (!isWaitingForFloatingBuffers) {
    +			throw new IllegalStateException("This channel should be waiting for floating buffers.");
    --- End diff --
    
    use `Preconditions.checkState()` here?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141380483
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandlerTest.java ---
    @@ -169,95 +182,19 @@ public void testCancelBeforeActive() throws Exception {
     		client.cancelRequestFor(inputChannel.getInputChannelId());
     	}
     
    -	/**
    -	 * Tests that an unsuccessful message decode call for a staged message
    -	 * does not leave the channel with auto read set to false.
    -	 */
    -	@Test
    -	@SuppressWarnings("unchecked")
    -	public void testAutoReadAfterUnsuccessfulStagedMessage() throws Exception {
    --- End diff --
    
    I will consider it


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152234362
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -287,17 +290,22 @@ void notifyCreditAvailable() {
     	}
     
     	/**
    -	 * Exclusive buffer is recycled to this input channel directly and it may trigger notify
    -	 * credit to producer.
    +	 * Exclusive buffer is recycled to this input channel directly and it may trigger return extra
    +	 * floating buffer and notify increased credit to the producer.
     	 *
     	 * @param segment The exclusive segment of this channel.
     	 */
     	@Override
     	public void recycle(MemorySegment segment) {
    -		synchronized (availableBuffers) {
    -			// Important: the isReleased check should be inside the synchronized block.
    -			// that way the segment can also be returned to global pool after added into
    -			// the available queue during releasing all resources.
    +		boolean floatingBufferRecycled = false;
    +
    +		synchronized (bufferQueue) {
    +			final int numRequiredBuffers = senderBacklog + initialCredit;
    +			checkState(bufferQueue.getAvailableBufferSize() <= numRequiredBuffers,
    +				"The number of available buffers " + bufferQueue.getAvailableBufferSize() + " should not exceed " + numRequiredBuffers);
    +
    +			// Important: check the isReleased state inside synchronized block, so there is no
    +			// race condition when recycle and releaseAllResources running in parallel.
     			if (isReleased.get()) {
     				try {
     					inputGate.returnExclusiveSegments(Arrays.asList(segment));
    --- End diff --
    
    now that I see it, it should rather be `inputGate.returnExclusiveSegments(Collections.singletonList(segment));`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152855823
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    --- End diff --
    
    add note that one exclusive buffer is taken (and therefore 11 requests and not 10)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152280350
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    --- End diff --
    
    Can you also verify the behaviour when the buffers become available? That is, create the `bufferPool` with `numFloatingBuffers+1` buffers initially, take one of them right after the creation, then continue as is and here `recycle()` this buffer so that the channel should get it via the buffer listener.
    Actually, it might even be useful to not only wait for 1 missing buffer, but to change the number so that is should receive 2 of them so that we test that the listener is actually getting not only one.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141902956
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -70,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    +
    +	/** The current available buffers including both exclusive buffers and requested floating buffers. */
    +	private final ArrayDeque<Buffer> availableBuffers = new ArrayDeque<>();
    +
    +	/** The number of available buffers that have not been announced to the producer yet. */
    +	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
    +
    +	/** The number of unsent buffers in the producer's sub partition. */
    +	private final AtomicInteger senderBacklog = new AtomicInteger(0);
    +
    +	/** The tag indicates whether this channel is waiting for additional floating buffers from the buffer pool. */
    +	private final AtomicBoolean isWaitingForFloatingBuffers = new AtomicBoolean(false);
    --- End diff --
    
    Now seeing this in action: do we really need a `AtomicBoolean`? Or is a `volatile boolean` enough? All uses except for `notifyBufferDestroyed()` (where only a safety-check uses the value) are actually under a `synchronized (availableBuffers)` block...in this case, you may also annotate the variable as `@GuardedBy("availableBuffers")` for documentation.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152255026
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -82,17 +84,19 @@
     	/** The initial number of exclusive buffers assigned to this channel. */
     	private int initialCredit;
     
    -	/** The current available buffers including both exclusive buffers and requested floating buffers. */
    -	private final ArrayDeque<Buffer> availableBuffers = new ArrayDeque<>();
    +	/** The available buffer queue wraps both exclusive and requested floating buffers. */
    +	private final AvailableBufferQueue bufferQueue = new AvailableBufferQueue();
     
     	/** The number of available buffers that have not been announced to the producer yet. */
     	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
     
     	/** The number of unsent buffers in the producer's sub partition. */
    -	private final AtomicInteger senderBacklog = new AtomicInteger(0);
    +	@GuardedBy("bufferQueue")
    +	private int senderBacklog;
    --- End diff --
    
    Skipping over the code, we only ever need `numRequiredBuffers = senderBacklog + initialCredit` - what do you think about storing this value instead of the backlog itself? It could still be retrieved if desired, but we spare adding these two numbers over and over again and only need to update this in `onSenderBacklog()`.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152852133
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    +			exclusiveBuffers.add(buffer);
    +
    +			if (getAvailableBufferSize() > numRequiredBuffers) {
    +				Buffer floatingBuffer = floatingBuffers.poll();
    +				floatingBuffer.recycle();
    +				return true;
    +			} else {
    +				return false;
    +			}
     		}
     
    -		int getFloatingBufferSize() {
    -			return floatingBuffers.size();
    +		/**
    +		 * Take the floating buffer first if possible.
    +		 */
    +		@Nullable
    +		Buffer takeBuffer() {
    --- End diff --
    
    please explain when the result may be `null`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW closed the pull request at:

    https://github.com/apache/flink/pull/4509


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141901569
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -209,6 +276,95 @@ public String toString() {
     	}
     
     	// ------------------------------------------------------------------------
    +	// Credit-based
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Enqueue this input channel in the pipeline for sending unannounced credits to producer.
    +	 */
    +	void notifyCreditAvailable() {
    +		//TODO in next PR
    +	}
    +
    +	/**
    +	 * Exclusive buffer is recycled to this input channel directly and it may trigger notify
    +	 * credit to producer.
    +	 *
    +	 * @param segment The exclusive segment of this channel.
    +	 */
    +	@Override
    +	public void recycle(MemorySegment segment) {
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			// that way the segment can also be returned to global pool after added into
    +			// the available queue during releasing all resources.
    +			if (isReleased.get()) {
    +				try {
    +					inputGate.returnExclusiveSegments(Arrays.asList(segment));
    +					return;
    +				} catch (Throwable t) {
    +					ExceptionUtils.rethrow(t);
    +				}
    +			}
    +			availableBuffers.add(new Buffer(segment, this));
    +		}
    +
    +		if (unannouncedCredit.getAndAdd(1) == 0) {
    +			notifyCreditAvailable();
    +		}
    +	}
    +
    +	public int getNumberOfAvailableBuffers() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.size();
    +		}
    +	}
    +
    +	/**
    +	 * The Buffer pool notifies this channel of an available floating buffer. If the channel is released or
    +	 * currently does not need extra buffers, the buffer should be recycled to the buffer pool. Otherwise,
    +	 * the buffer will be added into the <tt>availableBuffers</tt> queue and the unannounced credit is
    +	 * increased by one.
    +	 *
    +	 * @param buffer Buffer that becomes available in buffer pool.
    +	 * @return True when this channel is waiting for more floating buffers, otherwise false.
    +	 */
    +	@Override
    +	public boolean notifyBufferAvailable(Buffer buffer) {
    +		checkState(isWaitingForFloatingBuffers.get(), "This channel should be waiting for floating buffers.");
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (isReleased.get() || availableBuffers.size() >= senderBacklog.get()) {
    +				isWaitingForFloatingBuffers.set(false);
    +				buffer.recycle();
    +
    +				return false;
    +			}
    +
    +			availableBuffers.add(buffer);
    +
    +			if (unannouncedCredit.getAndAdd(1) == 0) {
    +				notifyCreditAvailable();
    +			}
    --- End diff --
    
    can we do this outside the `synchronized` block?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152290197
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -301,81 +306,388 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    -	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool
    +	 * in order to maintain backlog + initialCredit buffers available once receiving the
    +	 * sender's backlog, and registers as listener if no floating buffers available.
     	 */
     	@Test
    -	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    -		final SingleInputGate inputGate = mock(SingleInputGate.class);
    -		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +		// Setup
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final SingleInputGate inputGate = createSingleInputGate();
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		try {
    +			final int numFloatingBuffers = 10;
    +			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
    +			inputGate.setBufferPool(bufferPool);
    +
    +			// Assign exclusive segments to the channel
    +			final int numExclusiveBuffers = 2;
    +			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
    +			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
    +
    +			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    +				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
     
    -		// Recycle exclusive segment
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			// Receive the producer's backlog
    +			inputChannel.onSenderBacklog(8);
     
    -		assertEquals("There should be one buffer available after recycle.",
    -			1, inputChannel.getNumberOfAvailableBuffers());
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    +			verify(bufferPool, times(8)).requestBuffer();
    +			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
     
    -		inputChannel.recycle(MemorySegmentFactory.allocateUnpooledSegment(1024, inputChannel));
    +			inputChannel.onSenderBacklog(11);
     
    -		assertEquals("There should be two buffers available after recycle.",
    -			2, inputChannel.getNumberOfAvailableBuffers());
    -		// It should be called only once when increased from zero.
    -		verify(inputChannel, times(1)).notifyCreditAvailable();
    +			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +			assertEquals("There should be 12 buffers available in the channel",
    +				12, inputChannel.getNumberOfAvailableBuffers());
    +
    +			inputChannel.onSenderBacklog(12);
    +
    +			// Already in the status of waiting for buffers and will not request any more
    +			verify(bufferPool, times(11)).requestBuffer();
    +			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    +
    --- End diff --
    
    - Also, could you add further checks that verify that we stick to `senderBacklog + initialCredit` buffers after releasing (some of the) floating buffers, e.g. by getting them via `RemoteInputChannel#requestBuffer` and then `recycle()` them?
      - What happens when the sender backlog decreases? (that should work and release buffers accordingly)
      - Are extra floating buffers released correctly when we return exclusive buffers and have too many buffers?
    - You could also try to release the exclusive buffers the same way and ensure that they are recycled back into the channel and that we always maintain `initialCredit` buffers.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141664479
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java ---
    @@ -176,6 +176,8 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf msg, List<Object> out)
     
     		final int sequenceNumber;
     
    +		int backlog;
    --- End diff --
    
    please make this `final`


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152893854
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -475,16 +515,10 @@ public Void call() throws Exception {
     			};
     
     			// Submit tasks and wait to finish
    -			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    -			results.add(executor.submit(requestBufferTask));
    -			results.add(executor.submit(releaseTask));
    -			for (Future<Void> result : results) {
    -				result.get();
    -			}
    +			submitTasksAndWaitResults(executor, new Callable[]{requestBufferTask, releaseTask});
     
     			assertEquals("There should be no buffers available in the channel.",
     				0, inputChannel.getNumberOfAvailableBuffers());
    --- End diff --
    
    yes


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143225914
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,10 +311,27 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    +
    +			// Recycle the extra floating buffers in order not to stack up 2*initialCredit
    +			// buffers once current backlog is 0
    +			if (senderBacklog.get() == 0 && availableBuffers.size() >= initialCredit) {
    --- End diff --
    
    Actually, I guess, this should happen after adding the segment to `availableBuffers` and also not only when `senderBacklog.get() == 0`. The idiom you could follow is that after we integrated our exclusive buffer back, we should have a total of `buffersToMaintain = senderBacklog.get() + initialCredit` buffers. If there are more, we release `availableBuffers.size() - buffersToMaintain` _floating_ buffers accordingly. Since we only get one exclusive buffer back at a time, we only need to release at most one floating buffer here, as done by your code.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141292253
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(CreditBasedClientHandler.class);
    +
    +	private final ConcurrentMap<InputChannelID, RemoteInputChannel> inputChannels = new ConcurrentHashMap<>();
    +
    +	private final AtomicReference<Throwable> channelError = new AtomicReference<>();
    +
    +	/**
    +	 * Set of cancelled partition requests. A request is cancelled iff an input channel is cleared
    +	 * while data is still coming in for this channel.
    +	 */
    +	private final ConcurrentMap<InputChannelID, InputChannelID> cancelled = Maps.newConcurrentMap();
    +
    +	private volatile ChannelHandlerContext ctx;
    +
    +	// ------------------------------------------------------------------------
    +	// Input channel/receiver registration
    +	// ------------------------------------------------------------------------
    +
    +	void addInputChannel(RemoteInputChannel listener) throws IOException {
    +		checkError();
    +
    +		if (!inputChannels.containsKey(listener.getInputChannelId())) {
    +			inputChannels.put(listener.getInputChannelId(), listener);
    +		}
    +	}
    +
    +	void removeInputChannel(RemoteInputChannel listener) {
    +		inputChannels.remove(listener.getInputChannelId());
    +	}
    +
    +	void cancelRequestFor(InputChannelID inputChannelId) {
    +		if (inputChannelId == null || ctx == null) {
    +			return;
    +		}
    +
    +		if (cancelled.putIfAbsent(inputChannelId, inputChannelId) == null) {
    +			ctx.writeAndFlush(new NettyMessage.CancelPartitionRequest(inputChannelId));
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +	// Network events
    +	// ------------------------------------------------------------------------
    +
    +	@Override
    +	public void channelActive(final ChannelHandlerContext ctx) throws Exception {
    +		if (this.ctx == null) {
    +			this.ctx = ctx;
    +		}
    +
    +		super.channelActive(ctx);
    +	}
    +
    +	@Override
    +	public void channelInactive(ChannelHandlerContext ctx) throws Exception {
    +		// Unexpected close. In normal operation, the client closes the connection after all input
    +		// channels have been removed. This indicates a problem with the remote task manager.
    +		if (!inputChannels.isEmpty()) {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Connection unexpectedly closed by remote task manager '" + remoteAddr + "'. "
    +							+ "This might indicate that the remote task manager was lost.",
    +				remoteAddr));
    +		}
    +
    +		super.channelInactive(ctx);
    +	}
    +
    +	/**
    +	 * Called on exceptions in the client handler pipeline.
    +	 *
    +	 * <p> Remote exceptions are received as regular payload.
    +	 */
    +	@Override
    +	public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
    +
    +		if (cause instanceof TransportException) {
    +			notifyAllChannelsOfErrorAndClose(cause);
    +		} else {
    +			final SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			final TransportException tex;
    +
    +			// Improve on the connection reset by peer error message
    +			if (cause instanceof IOException
    +					&& cause.getMessage().equals("Connection reset by peer")) {
    +
    +				tex = new RemoteTransportException("Lost connection to task manager '" + remoteAddr + "'. " +
    +					"This indicates that the remote task manager was lost.", remoteAddr, cause);
    +			} else {
    +				tex = new LocalTransportException(cause.getMessage(), ctx.channel().localAddress(), cause);
    +			}
    +
    +			notifyAllChannelsOfErrorAndClose(tex);
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
    +		try {
    +			decodeMsg(msg);
    +		} catch (Throwable t) {
    +			notifyAllChannelsOfErrorAndClose(t);
    +		}
    +	}
    +
    +	private void notifyAllChannelsOfErrorAndClose(Throwable cause) {
    +		if (channelError.compareAndSet(null, cause)) {
    +			try {
    +				for (RemoteInputChannel inputChannel : inputChannels.values()) {
    +					inputChannel.onError(cause);
    +				}
    +			} catch (Throwable t) {
    +				// We can only swallow the Exception at this point. :(
    +				LOG.warn("An Exception was thrown during error notification of a remote input channel.", t);
    +			} finally {
    +				inputChannels.clear();
    +
    +				if (ctx != null) {
    +					ctx.close();
    +				}
    +			}
    +		}
    +	}
    +
    +	// ------------------------------------------------------------------------
    +
    +	/**
    +	 * Checks for an error and rethrows it if one was reported.
    +	 */
    +	private void checkError() throws IOException {
    +		final Throwable t = channelError.get();
    +
    +		if (t != null) {
    +			if (t instanceof IOException) {
    +				throw (IOException) t;
    +			} else {
    +				throw new IOException("There has been an error in the channel.", t);
    +			}
    +		}
    +	}
    +
    +	@Override
    +	public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
    +		super.channelReadComplete(ctx);
    +	}
    +
    +	private void decodeMsg(Object msg) throws Throwable {
    +		final Class<?> msgClazz = msg.getClass();
    +
    +		// ---- Buffer --------------------------------------------------------
    +		if (msgClazz == NettyMessage.BufferResponse.class) {
    +			NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg;
    +
    +			RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId);
    +			if (inputChannel == null) {
    +				bufferOrEvent.releaseBuffer();
    +
    +				cancelRequestFor(bufferOrEvent.receiverId);
    +
    +				return;
    +			}
    +
    +			decodeBufferOrEvent(inputChannel, bufferOrEvent);
    +
    +		} else if (msgClazz == NettyMessage.ErrorResponse.class) {
    +			// ---- Error ---------------------------------------------------------
    +			NettyMessage.ErrorResponse error = (NettyMessage.ErrorResponse) msg;
    +
    +			SocketAddress remoteAddr = ctx.channel().remoteAddress();
    +
    +			if (error.isFatalError()) {
    +				notifyAllChannelsOfErrorAndClose(new RemoteTransportException(
    +					"Fatal error at remote task manager '" + remoteAddr + "'.",
    +					remoteAddr,
    +					error.cause));
    +			} else {
    +				RemoteInputChannel inputChannel = inputChannels.get(error.receiverId);
    +
    +				if (inputChannel != null) {
    +					if (error.cause.getClass() == PartitionNotFoundException.class) {
    +						inputChannel.onFailedPartitionRequest();
    +					} else {
    +						inputChannel.onError(new RemoteTransportException(
    +							"Error at remote task manager '" + remoteAddr + "'.",
    +							remoteAddr,
    +							error.cause));
    +					}
    +				}
    +			}
    +		} else {
    +			throw new IllegalStateException("Received unknown message from producer: " + msg.getClass());
    +		}
    +	}
    +
    +	private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable {
    +		try {
    +			if (bufferOrEvent.isBuffer()) {
    +				// ---- Buffer ------------------------------------------------
    +
    +				// Early return for empty buffers. Otherwise Netty's readBytes() throws an
    +				// IndexOutOfBoundsException.
    +				if (bufferOrEvent.getSize() == 0) {
    +					inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber);
    +					return;
    +				}
    +
    +				Buffer buffer = inputChannel.requestBuffer();
    +				if (buffer != null) {
    +					buffer.setSize(bufferOrEvent.getSize());
    +					bufferOrEvent.getNettyBuffer().readBytes(buffer.getNioBuffer());
    +
    +					inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
    +					inputChannel.onSenderBacklog(bufferOrEvent.backlog);
    +				} else if (inputChannel.isReleased()) {
    +					cancelRequestFor(bufferOrEvent.receiverId);
    +				} else {
    +					throw new IOException("There should always have available buffer for credit-based.");
    --- End diff --
    
    `No buffer available in credit-based input channel. This should not happen and indicates an error in the communication protocol.`
    
    Maybe `IOException` is also not the right exception class for this but rather `IllegalStateException`?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152255421
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -394,7 +419,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode unless
    +	 * the channel has been released.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	@Nullable
    +	public Buffer requestBuffer() {
    +		synchronized (bufferQueue) {
    +			// Take the floating buffer first if possible.
    +			if (bufferQueue.getFloatingBufferSize() > 0) {
    --- End diff --
    
    Also extract this into a more generic `bufferQueue.takeBuffer()` and move the logic (exclusive vs. floating buffers) there?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141886467
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    --- End diff --
    
    I was thinking about it a bit more and was talking to @StephanEwen about it, and we think, that it is actually fine to grab all resources we need at the moment. If there are not enough buffers at some point, the fair distribution will start when the buffers are recycled, i.e. via the callbacks of the new `BufferListener`. Since each channel always has its own exclusive buffers, we can guarantee that it always makes progress anyway! Additionally, we cannot really make a fair distribution from the start when receiving the first backlog (since we do not know all the other backlogs) unless we're waiting some time which we also do not want.
    
    I kind of like your idea of having a `numBuffersPerAllocation`. Let's keep this in our heads and evaluate the current solution first to see whether we need this addition.
    
    Regarding the formula (which I took from the network FLIP):
    - from the FLIP with regards to the buffer distribution strategy: `Design rationale 2: Each channel always tries to maintain a credit of ‘backlog_length + initialCredit’. That means that each channel tries to build the receive window for its current backlog as much as possible from the floating buffers, and use the exclusive ‘initialCredit’ buffers as a means to grow the window.` That way we always have some buffers available immediately on the receiver side so the sender can continue sending new buffers immediately (as long as there are buffers available on the receiver) and we do not have to wait for the exclusive buffers to come back.
      - Note that this would have to be changed in the various checks for `availableBuffers.size() >= senderBacklog.get()`, e.g. in `RemoteInputChannel#notifyBufferAvailable()`.
      - Similarly, `RemoteInputChannel#recycle()` needs to be adapted, in case our exclusive buffers are in use and we requested `backlog_length + initialCredit - currentCredit` *floating* buffers in order not to stack up `2*initialCredit` buffers once `backlog == 0` again. (+ an according unit test)
    - what do you mean with `backlog-currentCredit` not being very accurate? We guarantee that there are no more than `currentCredit` buffers on the wire (some alraedy in the channel, some only announced) and, at the time the buffer was sent, `backlog` additional buffers were queued so in order to send them, we always need `backlog-currentCredit` irrespective of how much credit is announced vs. being on the wire. Or am I not seeing something here?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141794461
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    +					Buffer buffer = null;
    +					try {
    +						buffer = inputGate.getBufferPool().requestBuffer();
    +					} catch (IOException ex) {
    --- End diff --
    
    If throw `IOException` directly here, `onSenderBacklog()` and `onBuffer()` methods should both declare to throw this exception, and all the related upper methods have to handle this exception.
    
    I will consider this issue and may modify it along time the distribution of floating buffers above.
    BTW, all the other comments have been fixed.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143680495
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,10 +311,27 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    +
    +			// Recycle the extra floating buffers in order not to stack up 2*initialCredit
    +			// buffers once current backlog is 0
    +			if (senderBacklog.get() == 0 && availableBuffers.size() >= initialCredit) {
    +				final int size = availableBuffers.size();
    +				for (int i = 0; i < size; i++) {
    +					final Buffer buffer = availableBuffers.poll();
    --- End diff --
    
    I will consider this suggestion further.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by bowenli86 <gi...@git.apache.org>.
Github user bowenli86 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141945843
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java ---
    @@ -235,14 +240,15 @@ void releaseBuffer() {
     		ByteBuf write(ByteBufAllocator allocator) throws IOException {
     			checkNotNull(buffer, "No buffer instance to serialize.");
     
    -			int length = 16 + 4 + 1 + 4 + buffer.getSize();
    +			int length = 16 + 4 + 4 + 1 + 4 + buffer.getSize();
    --- End diff --
    
    can you please add a comment here explaining reasons of the math?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143422899
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -374,6 +376,64 @@ public void testReleaseExclusiveBuffers() throws Exception {
     		verify(inputGate, times(1)).returnExclusiveSegments(anyListOf(MemorySegment.class));
     	}
     
    +	/**
    +	 * Tests {@link BufferPool#requestBuffer()}, verifying the remote input channel tries to request
    +	 * floating buffers once receiving the producer's backlog.
    +	 */
    +	@Test
    +	public void testRequestFloatingBuffersOnBuffer() throws Exception {
    +		// Setup
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(TestBufferFactory.createBuffer());
    +
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		when(inputGate.getBufferPool()).thenReturn(bufferPool);
    +
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +
    +		// Receive the producer's backlog
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 0, 10);
    +		// Need to request 10 floating buffers from buffer pool
    +		verify(bufferPool, times(10)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 1, 8);
    +		// No need to request extra floating buffers from pool because
    +		// there are already 10 available buffers in queue now
    +		verify(bufferPool, times(10)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 2, 11);
    +		// Need to request another floating buffer from pool
    +		verify(bufferPool, times(11)).requestBuffer();
    +	}
    +
    +	/**
    +	 * Tests {@link BufferPool#requestBuffer()}, verifying the remote input channel tries to request
    +	 * floating buffers once receiving the producer's backlog. And it requests from pool only once
    +	 * and registers as listener if there are currently no available buffers in the pool.
    +	 */
    +	@Test
    +	public void testWaitForFloatingBuffersOnBuffer() throws Exception {
    +		// Setup
    +		final BufferPool bufferPool = mock(BufferPool.class);
    +		when(bufferPool.requestBuffer()).thenReturn(null);
    +		when(bufferPool.addBufferListener(any(BufferListener.class))).thenReturn(true);
    +
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		when(inputGate.getBufferPool()).thenReturn(bufferPool);
    +		when(inputGate.getBufferProvider()).thenReturn(bufferPool);
    +
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +
    +		// Receive the producer's backlog
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 0, 10);
    +		// Request from pool only once if there are no available floating buffers
    +		verify(bufferPool, times(1)).requestBuffer();
    +
    +		inputChannel.onBuffer(TestBufferFactory.createBuffer(), 1, 10);
    +		// Already registers as listener to wait for notifications and will not request any more
    +		verify(bufferPool, times(1)).requestBuffer();
    +	}
    +
    --- End diff --
    
    * I guess, you can add the "fair buffer distribution" test in this PR, to reduce the rebasing efforts and not switching back and forth between PRs too much


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    @NicoK , the codes are updated.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141890624
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -390,7 +390,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	public Buffer requestBuffer() {
    +		synchronized (availableBuffers) {
    +			return availableBuffers.poll();
    +		}
    +	}
    +
    +	/**
    +	 * Receives the backlog from the producer's buffer response. If the number of available
    +	 * buffers is less than the backlog length, it will request floating buffers from the buffer
    +	 * pool, and then notify unannounced credits to the producer.
    +	 *
    +	 * @param backlog The number of unsent buffers in the producer's sub partition.
    +	 */
    +	private void onSenderBacklog(int backlog) {
    +		int numRequestedBuffers = 0;
    +
    +		synchronized (availableBuffers) {
    +			// Important: the isReleased check should be inside the synchronized block.
    +			if (!isReleased.get()) {
    +				senderBacklog.set(backlog);
    +
    +				while (senderBacklog.get() > availableBuffers.size() && !isWaitingForFloatingBuffers.get()) {
    +					Buffer buffer = null;
    +					try {
    +						buffer = inputGate.getBufferPool().requestBuffer();
    +					} catch (IOException ex) {
    +						ExceptionUtils.rethrow(ex);
    +					}
    +
    +					if (buffer != null) {
    +						availableBuffers.add(buffer);
    +						numRequestedBuffers++;
    +						continue;
    +					}
    +
    +					// If the channel has not got enough buffers, register it as listener to wait for more floating buffers.
    +					if (inputGate.getBufferProvider().addBufferListener(this)) {
    +						if (!isWaitingForFloatingBuffers.compareAndSet(false, true)) {
    +							throw new IllegalStateException("This channel should not be waiting for floating buffers.");
    +						}
    --- End diff --
    
    ...instead, here you can actively `break` out of the loop early without having to check the conditions again


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r141376844
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedClientHandler.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.netty;
    +
    +import org.apache.flink.core.memory.MemorySegment;
    +import org.apache.flink.core.memory.MemorySegmentFactory;
    +import org.apache.flink.runtime.io.network.buffer.Buffer;
    +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
    +import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException;
    +import org.apache.flink.runtime.io.network.netty.exception.TransportException;
    +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException;
    +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID;
    +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
    +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.SocketAddress;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +class CreditBasedClientHandler extends ChannelInboundHandlerAdapter {
    --- End diff --
    
    sure


---

[GitHub] flink issue #4509: [FLINK-7406][network] Implement Netty receiver incoming p...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on the issue:

    https://github.com/apache/flink/pull/4509
  
    Already close it


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by zhijiangW <gi...@git.apache.org>.
Github user zhijiangW commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152456861
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -394,7 +419,63 @@ public BufferProvider getBufferProvider() throws IOException {
     		return inputGate.getBufferProvider();
     	}
     
    -	public void onBuffer(Buffer buffer, int sequenceNumber) {
    +	/**
    +	 * Requests buffer from input channel directly for receiving network data.
    +	 * It should always return an available buffer in credit-based mode unless
    +	 * the channel has been released.
    +	 *
    +	 * @return The available buffer.
    +	 */
    +	@Nullable
    +	public Buffer requestBuffer() {
    +		synchronized (bufferQueue) {
    +			// Take the floating buffer first if possible.
    +			if (bufferQueue.getFloatingBufferSize() > 0) {
    --- End diff --
    
    that is fine


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r143424259
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -306,10 +311,27 @@ public void recycle(MemorySegment segment) {
     					ExceptionUtils.rethrow(t);
     				}
     			}
    +
    +			// Recycle the extra floating buffers in order not to stack up 2*initialCredit
    +			// buffers once current backlog is 0
    +			if (senderBacklog.get() == 0 && availableBuffers.size() >= initialCredit) {
    --- End diff --
    
    This behaviour, i.e. sticking to `senderBacklog.get() + initialCredit` buffers in any case, should then also receive verification via a unit test :)


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152853208
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -572,20 +560,53 @@ void addExclusiveBuffer(Buffer buffer) {
     			exclusiveBuffers.add(buffer);
     		}
     
    -		Buffer takeExclusiveBuffer() {
    -			return exclusiveBuffers.poll();
    -		}
    -
     		void addFloatingBuffer(Buffer buffer) {
     			floatingBuffers.add(buffer);
     		}
     
    -		Buffer takeFloatingBuffer() {
    -			return floatingBuffers.poll();
    +		/**
    +		 * Add the exclusive buffer into the queue, and recycle one floating buffer if the
    +		 * number of available buffers in queue is more than required amount.
    +		 *
    +		 * @param buffer The exclusive buffer of this channel.
    +		 * @return Whether to recycle one floating buffer.
    +		 */
    +		boolean maintainTargetSize(Buffer buffer) {
    --- End diff --
    
    actually, this is now offering two functionalities:
    ```
    addExclusiveBuffer(buffer);
    maintainTargetSize(numRequiredBuffers);
    ```
    I'd suggest to either use the two separately or integrate the target size maintaining into `addExclusiveBuffers`. In any case, you should make `AvailableBufferQueue` a `static` inner class which could then also be tested separately.


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152969860
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -330,64 +332,120 @@ public void testRequestAndReturnFloatingBuffer() throws Exception {
     			// Prepare the exclusive and floating buffers to verify recycle logic later
     			Buffer exclusiveBuffer = inputChannel.requestBuffer();
     			assertNotNull(exclusiveBuffer);
    -			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    -			assertNotNull(floatingBuffer1);
    -			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    -			assertNotNull(floatingBuffer2);
    +
    +			final int numRecycleFloatingBuffers = 4;
    +			final ArrayDeque<Buffer> floatingBufferQueue = new ArrayDeque<>(numRecycleFloatingBuffers);
    +			for (int i = 0; i < numRecycleFloatingBuffers; i++) {
    +				Buffer floatingBuffer = bufferPool.requestBuffer();
    +				assertNotNull(floatingBuffer);
    +				floatingBufferQueue.add(floatingBuffer);
    +			}
     
     			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers.
    +			// One exclusive buffer is taken before, so we should request 13 floating buffers.
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 10 buffers available in the channel",
    +				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel",
    +				10, inputChannel.getNumberOfRequiredBuffers());
     
     			// Increase the backlog to exceed the number of available floating buffers
     			inputChannel.onSenderBacklog(10);
     
     			// The channel does not get enough floating buffer and register as buffer listener
    -			verify(bufferPool, times(13)).requestBuffer();
    +			verify(bufferPool, times(15)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			assertEquals("There should be 11 buffers available in the channel",
    +				11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 12 buffers required in the channel",
    +				12, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
     
     			// Continue increasing the backlog
    -			inputChannel.onSenderBacklog(11);
    +			inputChannel.onSenderBacklog(12);
     
     			// The channel is already in the status of waiting for buffers and will not request any more
    -			verify(bufferPool, times(13)).requestBuffer();
    +			verify(bufferPool, times(15)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			assertEquals("There should be 11 buffers available in the channel",
    +				11, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 14 buffers required in the channel",
    +				14, inputChannel.getNumberOfRequiredBuffers());
    +			assertEquals("There should be 0 buffer available in local pool",
    +				0, bufferPool.getNumberOfAvailableMemorySegments());
     
    -			// Recycle the floating buffer and assign it to the buffer listener
    -			floatingBuffer1.recycle();
    +			// Recycle one floating buffer
    +			floatingBufferQueue.poll().recycle();
     
    -			// The channel is still waiting for one more floating buffer
    -			assertEquals("There should be 12 buffers available in the channel", 12, inputChannel.getNumberOfAvailableBuffers());
    -			assertEquals("There should be 13 buffers required in the channel", 13, inputChannel.getNumberOfRequiredBuffers());
    -			assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments());
    +			// Assign the floating buffer to the listener and the channel is still waiting for more floating buffers
    +			assertEquals("There should be 12 buffers available in the channel",
    --- End diff --
    
    we should also add
    ```
    			verify(bufferPool, times(15)).requestBuffer();
    			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    ```
    here and into all blocks of checks below to ensure that the behaviour is as expected


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152854762
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -306,51 +306,88 @@ public void testProducerFailedException() throws Exception {
     	}
     
     	/**
    -	 * Tests to verify that the input channel requests floating buffers from buffer pool
    -	 * in order to maintain backlog + initialCredit buffers available once receiving the
    -	 * sender's backlog, and registers as listener if no floating buffers available.
    +	 * Tests to verify that the input channel requests floating buffers from buffer pool for
    +	 * maintaining (backlog + initialCredit) available buffers once receiving the sender's backlog.
    +	 *
    +	 * <p>Verifies the logic of recycling floating buffer back into the input channel and the logic
    +	 * of returning extra floating buffer into the buffer pool during recycling exclusive buffer.
     	 */
     	@Test
    -	public void testRequestFloatingBufferOnSenderBacklog() throws Exception {
    +	public void testRequestAndReturnFloatingBuffer() throws Exception {
     		// Setup
    -		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32, MemoryType.HEAP);
    +		final NetworkBufferPool networkBufferPool = new NetworkBufferPool(14, 32, MemoryType.HEAP);
    +		final int numExclusiveBuffers = 2;
    +		final int numFloatingBuffers = 12;
    +
     		final SingleInputGate inputGate = createSingleInputGate();
     		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate);
    +		inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     		try {
    -			final int numFloatingBuffers = 10;
     			final BufferPool bufferPool = spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers));
     			inputGate.setBufferPool(bufferPool);
    -
    -			// Assign exclusive segments to the channel
    -			final int numExclusiveBuffers = 2;
    -			inputGate.setInputChannel(inputChannel.partitionId.getPartitionId(), inputChannel);
     			inputGate.assignExclusiveSegments(networkBufferPool, numExclusiveBuffers);
     
    -			assertEquals("There should be " + numExclusiveBuffers + " buffers available in the channel",
    -				numExclusiveBuffers, inputChannel.getNumberOfAvailableBuffers());
    +			// Prepare the exclusive and floating buffers to verify recycle logic later
    +			Buffer exclusiveBuffer = inputChannel.requestBuffer();
    +			assertNotNull(exclusiveBuffer);
    +			Buffer floatingBuffer1 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer1);
    +			Buffer floatingBuffer2 = bufferPool.requestBuffer();
    +			assertNotNull(floatingBuffer2);
     
    -			// Receive the producer's backlog
    +			// Receive the producer's backlog less than the number of available floating buffers
     			inputChannel.onSenderBacklog(8);
     
    -			// Request the number of floating buffers by the formula of backlog + initialCredit - availableBuffers
    -			verify(bufferPool, times(8)).requestBuffer();
    +			// Request the floating buffers to maintain (backlog + initialCredit) available buffers
    +			verify(bufferPool, times(11)).requestBuffer();
     			verify(bufferPool, times(0)).addBufferListener(inputChannel);
    -			assertEquals("There should be 10 buffers available in the channel",
    -				10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers available in the channel", 10, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 10 buffers required in the channel", 10, inputChannel.getNumberOfRequiredBuffers());
     
    -			inputChannel.onSenderBacklog(11);
    +			// Increase the backlog to exceed the number of available floating buffers
    +			inputChannel.onSenderBacklog(10);
     
    -			// Need extra three floating buffers, but only two buffers available in buffer pool, register as listener as a result
    -			verify(bufferPool, times(11)).requestBuffer();
    +			// The channel does not get enough floating buffer and register as buffer listener
    +			verify(bufferPool, times(13)).requestBuffer();
     			verify(bufferPool, times(1)).addBufferListener(inputChannel);
    -			assertEquals("There should be 12 buffers available in the channel",
    -				12, inputChannel.getNumberOfAvailableBuffers());
    +			assertEquals("There should be 11 buffers available in the channel", 11, inputChannel.getNumberOfAvailableBuffers());
    --- End diff --
    
    can you break those extra long lines?


---

[GitHub] flink pull request #4509: [FLINK-7406][network] Implement Netty receiver inc...

Posted by NicoK <gi...@git.apache.org>.
Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4509#discussion_r152862469
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -475,16 +515,10 @@ public Void call() throws Exception {
     			};
     
     			// Submit tasks and wait to finish
    -			final List<Future<Void>> results = Lists.newArrayListWithCapacity(2);
    -			results.add(executor.submit(requestBufferTask));
    -			results.add(executor.submit(releaseTask));
    -			for (Future<Void> result : results) {
    -				result.get();
    -			}
    +			submitTasksAndWaitResults(executor, new Callable[]{requestBufferTask, releaseTask});
     
     			assertEquals("There should be no buffers available in the channel.",
     				0, inputChannel.getNumberOfAvailableBuffers());
    --- End diff --
    
    please add:
    ```
    			assertEquals("There should be 130 buffers available in local pool.",
    				130, bufferPool.getNumberOfAvailableMemorySegments() + networkBufferPool.getNumberOfAvailableMemorySegments());
    ```
    (buffers could be in either buffer pool depending on whether they were requested at least once from the `LocalBufferPool` or not)


---