[GitHub] flink pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...
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 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...
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 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...
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...
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 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...
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 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 pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...
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 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] Implement basic InputChannel fo...
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 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] Implement basic InputChannel fo...
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 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...
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 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 pull request #4499: [FLINK-7394][core] Implement basic InputChannel fo...
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 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...
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 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] Implement basic InputChannel fo...
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] Implement basic InputChannel fo...
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> registeredListeners = new ArrayDeque>(); + private final Queue 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] Implement basic InputChannel fo...
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()`? ---
[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 Date: 2017-08-07T09:31:17Z [FLINK-7378][core]Implement the FixedBufferPool for floating buffers of SingleInputGate commit b3cabd92051b2682c869742a5a346193b3d4ca33 Author: Zhijiang 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---