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/09 06:53:33 UTC

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

GitHub user zhijiangW opened a pull request:

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

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

    ## What is the purpose of the change
    
    This pull request prepares the basic works for credit-based interaction with netty pipeline which will be submitted in next pull request.
    
    This pull request depends on previous [FixedBufferPool PR](https://github.com/apache/flink/pull/4485) whose commit is also included for passing travis, so review the second commit for this pull request change.
    
    In order not to affect the current process and existing cases, a temporary method called `testRegisterTask` is added in `NetworkEnvironment` for verifying partial behaviors until the whole feature codes are submitted.
    
    ## Brief change log
    
      - *`NetworkEnvironment` creates the `FixedBufferPool` for `SingleInputGate`*
      - *Assigns the exclusive buffers directly to `RemoteInputChannel` during registering task*
      - *`RemoteInputChannel` implements `BufferRecycler` interface to manage the exclusive buffers itself*
      - *`RemoteInputChannel` implements `BufferPoolListener` interface to be notified available floating buffers from buffer pool*
      - *`RemoteInputChannel` maintains unannounced credit and current sender backlog*
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
      - *Added test for registering task with FixedBufferPool creation and exclusive buffers assignment*
      - *The credit and backlog logics will be verified after implementing the netty pipeline part in the next pull request*
    
    ## 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): (yes)
      - 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-7394

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

    https://github.com/apache/flink/pull/4499.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 #4499
    
----
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 b3cabd92051b2682c869742a5a346193b3d4ca33
Author: Zhijiang <wa...@aliyun.com>
Date:   2017-08-09T05:43:56Z

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

----


---
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 #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138923001
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -183,18 +214,40 @@ void notifySubpartitionConsumed() {
     	}
     
     	/**
    -	 * Releases all received buffers and closes the partition request client.
    +	 * Releases all received and available buffers, closes the partition request client.
     	 */
     	@Override
     	void releaseAllResources() throws IOException {
     		if (isReleased.compareAndSet(false, true)) {
    +
    +			final List<MemorySegment> recyclingSegments = new ArrayList<>();
    +
     			synchronized (receivedBuffers) {
     				Buffer buffer;
     				while ((buffer = receivedBuffers.poll()) != null) {
    -					buffer.recycle();
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    +					} else {
    +						buffer.recycle();
    +					}
     				}
     			}
     
    +			synchronized (availableBuffers) {
    +				Buffer buffer;
    +				while ((buffer = availableBuffers.poll()) != null) {
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    --- End diff --
    
    same here


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r139589040
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    --- End diff --
    
    This variable is set in `assignExclusiveSegments` and will be read via sending `PartitionRequest` in following PR.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r139590215
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -100,7 +122,16 @@ public RemoteInputChannel(
     	}
     
     	void assignExclusiveSegments(List<MemorySegment> segments) {
    --- End diff --
    
    Yes, this method should be called only once after `RemoteInputChannel` created. I think  `assignExclusiveSegments` can describe the semantics of invoking only once, and `addExclusiveSegments` seems allow to be called multiple times.
    
    I currently take the third way you suggested, adding the `checkState` to avoid multi calling.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r136056661
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    +
    +	/** The current available exclusive buffers and requested floating buffers from buffer pool. */
    +	private final Queue<Buffer> availableBuffers = new ArrayDeque<>();
    +
    +	/** The number of available buffers that have not unannounced to producer yet. */
    +	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
    +
    +	/** The number of unsent buffers in producer's sub partition. */
    +	private final AtomicInteger currentSenderBacklog = new AtomicInteger(0);
    +
    +	/** The tag indicates whether this channel is waiting additional floating buffers from buffer pool. */
    --- End diff --
    
    `...waiting for additional floating buffers from the buffer pool`


---

[GitHub] flink issue #4499: [FLINK-7394][core] Manage exclusive buffers in RemoteInpu...

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

    https://github.com/apache/flink/pull/4499
  
    @NicoK , I have fixed the above issues you mentioned. I should double check the typos after submit next time.
    
    I agree and understand your feeling of `RemoteInputChannel#releaseAllResources()` which seems destroy the beauty of recycle. I would consider this issue and try to find other options if possible later. And I think this will not block the reviews of following PRs. :)


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140833417
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -281,6 +286,73 @@ public void testProducerFailedException() throws Exception {
     		ch.getNextBuffer();
     	}
     
    +	/**
    +	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    +	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 */
    +	@Test
    +	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +
    +		// Recycle exclusive segment
    +		inputChannel.recycle(MemorySegmentFactory.getFactory().allocateUnpooledSegment(1024, inputChannel));
    +
    +		assertEquals("There should have one available buffer after recycle.",
    +			1, inputChannel.getNumberOfAvailableBuffers());
    +		verify(inputChannel, times(1)).notifyCreditAvailable();
    --- End diff --
    
    can you add one more `inputChannel.recycle(MemorySegmentFactory.getFactory().allocateUnpooledSegment(1024, inputChannel))` call and verify `inputChannel.getNumberOfAvailableBuffers()` and that `notifyCreditAvailable()` is not called again?


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r136056467
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    +
    +	/** The current available exclusive buffers and requested floating buffers from buffer pool. */
    +	private final Queue<Buffer> availableBuffers = new ArrayDeque<>();
    --- End diff --
    
    I'd prefer to use `ArrayDeque` here for the member as well to help the JVM to optimise a bit (methods should mostly use less concrete types such as `Queue` to not limit users too much, but this is not necessary for a private member)


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140809272
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -70,6 +78,15 @@
     	 */
     	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 unannounced to producer yet. */
    --- End diff --
    
    actually, there were some more typos here - the full comment should be something like this `The number of available buffers that have not been announced to the producer yet.`


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140810853
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -99,8 +114,24 @@ public RemoteInputChannel(
     		this.connectionManager = checkNotNull(connectionManager);
     	}
     
    +	/**
    +	 * Assigns exclusive buffers to this input channel, and this method should be called only once
    +	 * after this input channel is created.
    +	 */
     	void assignExclusiveSegments(List<MemorySegment> segments) {
    -		// TODO in next PR
    +		checkState(this.initialCredit == 0, "Bug in input channel setup logic: exclusive buffers have" +
    +			"already been set for this input channel.");
    --- End diff --
    
    please add a space between `have` and `already` (between the concatenations)


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140821090
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -183,18 +214,40 @@ void notifySubpartitionConsumed() {
     	}
     
     	/**
    -	 * Releases all received buffers and closes the partition request client.
    +	 * Releases all received and available buffers, closes the partition request client.
     	 */
     	@Override
     	void releaseAllResources() throws IOException {
     		if (isReleased.compareAndSet(false, true)) {
    +
    +			final List<MemorySegment> recyclingSegments = new ArrayList<>();
    +
     			synchronized (receivedBuffers) {
     				Buffer buffer;
     				while ((buffer = receivedBuffers.poll()) != null) {
    -					buffer.recycle();
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    --- End diff --
    
    1) I think, performance is not much of an issue here, as this is only called during take-down of a connection and the overhead of `Buffer#recycle` is actually not that much.
    2) Sorry, but I don't get your concern. Why would you need an extra check when using `buffer.recycle()` instead of `exclusiveRecyclingSegments.add(buffer.getMemorySegment())`? There shouldn't be anything special for the exclusive buffers in this regard compared to ordinary buffers (which is the beauty of the design).
    
    Let me give the example of how `LocalBufferPool` handles this inside `lazyDestroy`: it returns every memory segment (one by one) with `networkBufferPool.recycle()` and, at its end, it is calling `networkBufferPool.destroyBufferPool()` so that the book-keeping inside the `NetworkBufferPool` is updated and buffers may be re-distributed to other `LocalBufferPool` instances.
    We could do this similarly here: recycle one by one, then call some method to update book-keeping and balancing inside `NetworkBufferPool`.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140811033
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -100,7 +122,16 @@ public RemoteInputChannel(
     	}
     
     	void assignExclusiveSegments(List<MemorySegment> segments) {
    --- End diff --
    
    works for me :)


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138909210
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java ---
    @@ -64,7 +63,7 @@
     	 * Buffer availability listeners, which need to be notified when a Buffer becomes available.
     	 * Listeners can only be registered at a time/state where no Buffer instance was available.
     	 */
    -	private final Queue<EventListener<Buffer>> registeredListeners = new ArrayDeque<EventListener<Buffer>>();
    +	private final Queue<BufferListener> registeredListeners = new ArrayDeque<>();
    --- End diff --
    
    I'd prefer to use `ArrayDeque` here for the member as well to help the JVM to optimise a bit (methods should mostly use less concrete types such as Queue to not limit users too much, but this is not necessary for a private member). 
    
    I don't have strong feelings about this though.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140831859
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -281,6 +286,73 @@ public void testProducerFailedException() throws Exception {
     		ch.getNextBuffer();
     	}
     
    +	/**
    +	 * Tests {@link RemoteInputChannel#recycle(MemorySegment)}, verifying the exclusive segment is
    +	 * recycled to available buffers directly and it triggers notify of announced credit.
    +	 */
    +	@Test
    +	public void testRecycleExclusiveBufferBeforeReleased() throws Exception {
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		final RemoteInputChannel inputChannel = spy(createRemoteInputChannel(inputGate));
    +
    +		// Recycle exclusive segment
    +		inputChannel.recycle(MemorySegmentFactory.getFactory().allocateUnpooledSegment(1024, inputChannel));
    +
    +		assertEquals("There should have one available buffer after recycle.",
    --- End diff --
    
    `There should be one buffer available after recycle`


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140829015
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -215,6 +269,48 @@ 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) {
    +			if (isReleased.get()) {
    +				try {
    +					inputGate.returnExclusiveSegments(Arrays.asList(segment));
    +					return;
    +				} catch (Throwable t) {
    +					ExceptionUtils.rethrow(t);
    +				}
    +			}
    --- End diff --
    
    Can you maybe add a comment on the importance of the `isReleased` check being inside the synchronized block (as implemented by `onBuffer` before, but also without a comment)? This is related to the `AtomicBoolean` field and not getting into races here.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138916859
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    --- End diff --
    
    Do we need this variable? In the current PR, it is not read anywhere.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140831302
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -183,18 +214,40 @@ void notifySubpartitionConsumed() {
     	}
     
     	/**
    -	 * Releases all received buffers and closes the partition request client.
    +	 * Releases all received and available buffers, closes the partition request client.
     	 */
     	@Override
     	void releaseAllResources() throws IOException {
     		if (isReleased.compareAndSet(false, true)) {
    +
    +			final List<MemorySegment> recyclingSegments = new ArrayList<>();
    +
     			synchronized (receivedBuffers) {
     				Buffer buffer;
     				while ((buffer = receivedBuffers.poll()) != null) {
    -					buffer.recycle();
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    --- End diff --
    
    ok - regarding `RemoteInputChannel#recycle()`, if we use this here (indirectly via `buffer.recycle`), we would not want it to redistribute buffers for every single `recycle()` call, but if `RemoteInputChannel#recycle()` is called for a single buffer outside `RemoteInputChannel#releaseAllResources()`, we would want it to do that...
    
    You're right that this would create some mess there and we should probably keep your current solution unless something better comes to mind.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138915141
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    +
    +	/** The current available exclusive buffers and requested floating buffers from buffer pool. */
    +	private final Queue<Buffer> availableBuffers = new ArrayDeque<>();
    +
    +	/** The number of available buffers that have not unannounced to producer yet. */
    --- End diff --
    
    `...have not been announced to the producer yet.`


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138922984
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -183,18 +214,40 @@ void notifySubpartitionConsumed() {
     	}
     
     	/**
    -	 * Releases all received buffers and closes the partition request client.
    +	 * Releases all received and available buffers, closes the partition request client.
     	 */
     	@Override
     	void releaseAllResources() throws IOException {
     		if (isReleased.compareAndSet(false, true)) {
    +
    +			final List<MemorySegment> recyclingSegments = new ArrayList<>();
    +
     			synchronized (receivedBuffers) {
     				Buffer buffer;
     				while ((buffer = receivedBuffers.poll()) != null) {
    -					buffer.recycle();
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    --- End diff --
    
    Although this is probably more performant than calling `recycle()`, the beauty of having `this` as the recycler for our exclusive buffers actually allows very simple and nice code only calling `recycle()` in all cases. Since we're not on the hot path here, I'd actually prefer that, or what do you think?


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

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


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138915417
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    +
    +	/** The current available exclusive buffers and requested floating buffers from buffer pool. */
    +	private final Queue<Buffer> availableBuffers = new ArrayDeque<>();
    +
    +	/** The number of available buffers that have not unannounced to producer yet. */
    +	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
    +
    +	/** The number of unsent buffers in producer's sub partition. */
    --- End diff --
    
    `...in the producer's...`


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Manage exclusive buffers in Rem...

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

    https://github.com/apache/flink/pull/4499#discussion_r140822508
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -183,18 +214,40 @@ void notifySubpartitionConsumed() {
     	}
     
     	/**
    -	 * Releases all received buffers and closes the partition request client.
    +	 * Releases all received and available buffers, closes the partition request client.
     	 */
     	@Override
     	void releaseAllResources() throws IOException {
     		if (isReleased.compareAndSet(false, true)) {
    +
    +			final List<MemorySegment> recyclingSegments = new ArrayList<>();
    +
     			synchronized (receivedBuffers) {
     				Buffer buffer;
     				while ((buffer = receivedBuffers.poll()) != null) {
    -					buffer.recycle();
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    --- End diff --
    
    oh, now I see - but let me elaborate on the `RemoteInputChannel#recycle()` there


---

[GitHub] flink issue #4499: [FLINK-7394][core] Implement basic InputChannel for credi...

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

    https://github.com/apache/flink/pull/4499
  
    @NicoK , thanks for your reviews!
    
    This PR mainly covers four processes related with credit-based feature in `RemoteInputChannel`. I think it is better to split this PR into several small ones, so we can focus on every process for easy review. Or what do you think?
    
    I will rebase this PR one the latest change of first PR and submit the updates today.


---

[GitHub] flink issue #4499: [FLINK-7394][core] Manage exclusive buffers in RemoteInpu...

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

    https://github.com/apache/flink/pull/4499
  
    merging.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r139596400
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -183,18 +214,40 @@ void notifySubpartitionConsumed() {
     	}
     
     	/**
    -	 * Releases all received buffers and closes the partition request client.
    +	 * Releases all received and available buffers, closes the partition request client.
     	 */
     	@Override
     	void releaseAllResources() throws IOException {
     		if (isReleased.compareAndSet(false, true)) {
    +
    +			final List<MemorySegment> recyclingSegments = new ArrayList<>();
    +
     			synchronized (receivedBuffers) {
     				Buffer buffer;
     				while ((buffer = receivedBuffers.poll()) != null) {
    -					buffer.recycle();
    +					if (buffer.getRecycler() == this) {
    +						recyclingSegments.add(buffer.getMemorySegment());
    --- End diff --
    
    I also considered this issue when implementation. The current way seems more verbose than directly calling `recycle()`, but there are two advantages:
    
    1. Exclusive buffers are recycled in batch which may be more performant as you mentioned above. 
    2. If calling `recycle()` directly, it should add extra check `isReleased` process outside the synchronized in `RemoteInputChannel#recycle` method.  It seems not elegant to do so. What do you think of this issue?



---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138918917
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -100,7 +122,16 @@ public RemoteInputChannel(
     	}
     
     	void assignExclusiveSegments(List<MemorySegment> segments) {
    --- End diff --
    
    This method is currently **adding** memory segments and multiple calls are allowed (making `initialCredit` reflect the newest call though). I know, we don't plan to call this multiple times, but there's also probably no harm. We should either
    
    1. rename the method to `addExclusiveSegments` and adapt the `initialCredit` (if we actually keep this), or
    2. implement the "assign" behaviour, i.e. recycle previously assigned elements and only use the new ones as exclusive buffers (for this you actually need the `initialCredit` but you may not be able to return previously added buffers immediately), or
    3. prevent the user from calling the method multiple times
    
    For simplicity, I'd go with number 1. What do you think?


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r138915958
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java ---
    @@ -72,6 +79,21 @@
     	 */
     	private int expectedSequenceNumber = 0;
     
    +	/** The initial number of exclusive buffers assigned to this channel. */
    +	private int initialCredit;
    +
    +	/** The current available exclusive buffers and requested floating buffers from buffer pool. */
    +	private final Queue<Buffer> availableBuffers = new ArrayDeque<>();
    +
    +	/** The number of available buffers that have not unannounced to producer yet. */
    +	private final AtomicInteger unannouncedCredit = new AtomicInteger(0);
    +
    +	/** The number of unsent buffers in producer's sub partition. */
    +	private final AtomicInteger currentSenderBacklog = new AtomicInteger(0);
    --- End diff --
    
    you could probably simplify the name to `senderBacklog`


---

[GitHub] flink issue #4499: [FLINK-7394][core] Implement basic InputChannel for credi...

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

    https://github.com/apache/flink/pull/4499
  
    @NicoK , I have submitted the updates based on the latest first PR. And this PR only concerns about the exclusive buffers management. I would submit the other issues in separate PRs later.


---

[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...

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

    https://github.com/apache/flink/pull/4499#discussion_r136027041
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java ---
    @@ -281,6 +284,67 @@ public void testProducerFailedException() throws Exception {
     		ch.getNextBuffer();
     	}
     
    +	@Test
    +	public void testNotifyCreditOnBufferRecycling() throws Exception {
    +		// Setup
    +		final SingleInputGate inputGate = mock(SingleInputGate.class);
    +		final PartitionRequestClient client = mock(PartitionRequestClient.class);
    +		final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate, client, new Tuple2<>(0, 0));
    +
    +		inputChannel.requestSubpartition(0);
    +
    +		// Recycle exclusive segment
    +		inputChannel.recycle(HeapMemorySegment.FACTORY.allocateUnpooledSegment(1024, inputChannel));
    --- End diff --
    
    why not use the more generic `MemorySegmentFactory.getFactory()`?


---