gaoyunhaii commented on a change in pull request #13924:
URL: https://github.com/apache/flink/pull/13924#discussion_r601153174



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java
##########
@@ -19,154 +19,180 @@
 package org.apache.flink.runtime.io.network.partition;
 
 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.BufferRecycler;
 import 
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.IOUtils;
 
 import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
 
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Queue;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
 
 /** Subpartition data reader for {@link SortMergeResultPartition}. */
-public class SortMergeSubpartitionReader implements ResultSubpartitionView, 
BufferRecycler {
+public class SortMergeSubpartitionReader
+        implements ResultSubpartitionView, 
Comparable<SortMergeSubpartitionReader> {
 
-    private static final int NUM_READ_BUFFERS = 2;
+    private final Object lock = new Object();
 
     /** Target {@link SortMergeResultPartition} to read data from. */
     private final SortMergeResultPartition partition;
 
     /** Listener to notify when data is available. */
     private final BufferAvailabilityListener availabilityListener;
 
-    /** Unmanaged memory used as read buffers. */
-    private final Queue<MemorySegment> readBuffers = new ArrayDeque<>();
-
-    /** Buffers read by the file reader. */
+    /** Buffers already read which can be consumed by netty thread. */
+    @GuardedBy("lock")
     private final Queue<Buffer> buffersRead = new ArrayDeque<>();
 
     /** File reader used to read buffer from. */
     private final PartitionedFileReader fileReader;
 
-    /** Number of remaining non-event buffers to read. */
+    /** Number of remaining non-event buffers in the buffer queue. */
+    @GuardedBy("lock")
     private int dataBufferBacklog;
 
     /** Whether this reader is released or not. */
+    @GuardedBy("lock")
     private boolean isReleased;
 
+    /** Cause of failure which should be propagated to the consumer. */
+    @GuardedBy("lock")
+    private Throwable failureCause;
+
     /** Sequence number of the next buffer to be sent to the consumer. */
     private int sequenceNumber;
 
     public SortMergeSubpartitionReader(
-            int subpartitionIndex,
-            int dataBufferBacklog,
-            int bufferSize,
             SortMergeResultPartition partition,
             BufferAvailabilityListener listener,
-            PartitionedFile partitionedFile)
-            throws IOException {
+            PartitionedFileReader fileReader) {
         this.partition = checkNotNull(partition);
         this.availabilityListener = checkNotNull(listener);
-        this.dataBufferBacklog = dataBufferBacklog;
-
-        // allocate two pieces of unmanaged segments for data reading
-        for (int i = 0; i < NUM_READ_BUFFERS; i++) {
-            this.readBuffers.add(
-                    
MemorySegmentFactory.allocateUnpooledOffHeapMemory(bufferSize, null));
-        }
-
-        this.fileReader = new PartitionedFileReader(partitionedFile, 
subpartitionIndex);
-        try {
-            readBuffers();
-        } catch (Throwable throwable) {
-            // ensure that the file reader is closed when any exception occurs
-            IOUtils.closeQuietly(fileReader);
-            throw throwable;
-        }
+        this.fileReader = checkNotNull(fileReader);
     }
 
     @Nullable
     @Override
     public BufferAndBacklog getNextBuffer() {
-        checkState(!isReleased, "Reader is already released.");
+        synchronized (lock) {
+            Buffer buffer = buffersRead.poll();
+            if (buffer == null) {
+                return null;
+            }
+
+            if (buffer.isBuffer()) {
+                --dataBufferBacklog;
+            }
 
-        Buffer buffer = buffersRead.poll();
-        if (buffer == null) {
-            return null;
+            Buffer lookAhead = buffersRead.peek();
+            return BufferAndBacklog.fromBufferAndLookahead(
+                    buffer,
+                    lookAhead == null ? Buffer.DataType.NONE : 
lookAhead.getDataType(),
+                    dataBufferBacklog,
+                    sequenceNumber++);
         }
+    }
 
-        if (buffer.isBuffer()) {
-            --dataBufferBacklog;
+    private void addBuffer(Buffer buffer) {
+        boolean notifyAvailable;
+        synchronized (lock) {
+            if (isReleased) {
+                buffer.recycleBuffer();
+                return;
+            }
+
+            notifyAvailable = buffersRead.isEmpty();
+
+            buffersRead.add(buffer);
+            if (buffer.isBuffer()) {
+                ++dataBufferBacklog;
+            }
         }
 
-        final Buffer lookAhead = buffersRead.peek();
+        if (notifyAvailable) {
+            notifyDataAvailable();
+        }
+    }
+
+    boolean readBuffers(Queue<MemorySegment> buffers, BufferRecycler recycler) 
throws IOException {
+        while (!buffers.isEmpty()) {

Review comment:
       Perhaps we could limit the maximum accumulated amount of buffers for a 
single sub-partition. But it should be an optimization and we could do it 
separately in the future after we have assessed the suitable threshold. 

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java
##########
@@ -19,154 +19,180 @@
 package org.apache.flink.runtime.io.network.partition;
 
 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.BufferRecycler;
 import 
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.IOUtils;
 
 import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
 
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Queue;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
 
 /** Subpartition data reader for {@link SortMergeResultPartition}. */
-public class SortMergeSubpartitionReader implements ResultSubpartitionView, 
BufferRecycler {
+public class SortMergeSubpartitionReader
+        implements ResultSubpartitionView, 
Comparable<SortMergeSubpartitionReader> {
 
-    private static final int NUM_READ_BUFFERS = 2;
+    private final Object lock = new Object();
 
     /** Target {@link SortMergeResultPartition} to read data from. */
     private final SortMergeResultPartition partition;
 
     /** Listener to notify when data is available. */
     private final BufferAvailabilityListener availabilityListener;
 
-    /** Unmanaged memory used as read buffers. */
-    private final Queue<MemorySegment> readBuffers = new ArrayDeque<>();
-
-    /** Buffers read by the file reader. */
+    /** Buffers already read which can be consumed by netty thread. */
+    @GuardedBy("lock")
     private final Queue<Buffer> buffersRead = new ArrayDeque<>();
 
     /** File reader used to read buffer from. */
     private final PartitionedFileReader fileReader;
 
-    /** Number of remaining non-event buffers to read. */
+    /** Number of remaining non-event buffers in the buffer queue. */
+    @GuardedBy("lock")
     private int dataBufferBacklog;
 
     /** Whether this reader is released or not. */
+    @GuardedBy("lock")
     private boolean isReleased;
 
+    /** Cause of failure which should be propagated to the consumer. */
+    @GuardedBy("lock")
+    private Throwable failureCause;
+
     /** Sequence number of the next buffer to be sent to the consumer. */
     private int sequenceNumber;
 
     public SortMergeSubpartitionReader(
-            int subpartitionIndex,
-            int dataBufferBacklog,
-            int bufferSize,
             SortMergeResultPartition partition,
             BufferAvailabilityListener listener,
-            PartitionedFile partitionedFile)
-            throws IOException {
+            PartitionedFileReader fileReader) {
         this.partition = checkNotNull(partition);
         this.availabilityListener = checkNotNull(listener);
-        this.dataBufferBacklog = dataBufferBacklog;
-
-        // allocate two pieces of unmanaged segments for data reading
-        for (int i = 0; i < NUM_READ_BUFFERS; i++) {
-            this.readBuffers.add(
-                    
MemorySegmentFactory.allocateUnpooledOffHeapMemory(bufferSize, null));
-        }
-
-        this.fileReader = new PartitionedFileReader(partitionedFile, 
subpartitionIndex);
-        try {
-            readBuffers();
-        } catch (Throwable throwable) {
-            // ensure that the file reader is closed when any exception occurs
-            IOUtils.closeQuietly(fileReader);
-            throw throwable;
-        }
+        this.fileReader = checkNotNull(fileReader);
     }
 
     @Nullable
     @Override
     public BufferAndBacklog getNextBuffer() {
-        checkState(!isReleased, "Reader is already released.");
+        synchronized (lock) {
+            Buffer buffer = buffersRead.poll();
+            if (buffer == null) {
+                return null;
+            }
+
+            if (buffer.isBuffer()) {
+                --dataBufferBacklog;
+            }
 
-        Buffer buffer = buffersRead.poll();
-        if (buffer == null) {
-            return null;
+            Buffer lookAhead = buffersRead.peek();
+            return BufferAndBacklog.fromBufferAndLookahead(
+                    buffer,
+                    lookAhead == null ? Buffer.DataType.NONE : 
lookAhead.getDataType(),
+                    dataBufferBacklog,
+                    sequenceNumber++);
         }
+    }
 
-        if (buffer.isBuffer()) {
-            --dataBufferBacklog;
+    private void addBuffer(Buffer buffer) {
+        boolean notifyAvailable;
+        synchronized (lock) {
+            if (isReleased) {
+                buffer.recycleBuffer();
+                return;
+            }
+
+            notifyAvailable = buffersRead.isEmpty();
+
+            buffersRead.add(buffer);
+            if (buffer.isBuffer()) {
+                ++dataBufferBacklog;
+            }
         }
 
-        final Buffer lookAhead = buffersRead.peek();
+        if (notifyAvailable) {
+            notifyDataAvailable();
+        }
+    }
+
+    boolean readBuffers(Queue<MemorySegment> buffers, BufferRecycler recycler) 
throws IOException {

Review comment:
       We may add a comment here on that this method is running in separate IO 
threads.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java
##########
@@ -19,154 +19,180 @@
 package org.apache.flink.runtime.io.network.partition;
 
 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.BufferRecycler;
 import 
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.IOUtils;
 
 import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
 
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Queue;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
 
 /** Subpartition data reader for {@link SortMergeResultPartition}. */
-public class SortMergeSubpartitionReader implements ResultSubpartitionView, 
BufferRecycler {
+public class SortMergeSubpartitionReader
+        implements ResultSubpartitionView, 
Comparable<SortMergeSubpartitionReader> {
 
-    private static final int NUM_READ_BUFFERS = 2;
+    private final Object lock = new Object();
 
     /** Target {@link SortMergeResultPartition} to read data from. */
     private final SortMergeResultPartition partition;
 
     /** Listener to notify when data is available. */
     private final BufferAvailabilityListener availabilityListener;
 
-    /** Unmanaged memory used as read buffers. */
-    private final Queue<MemorySegment> readBuffers = new ArrayDeque<>();
-
-    /** Buffers read by the file reader. */
+    /** Buffers already read which can be consumed by netty thread. */
+    @GuardedBy("lock")
     private final Queue<Buffer> buffersRead = new ArrayDeque<>();
 
     /** File reader used to read buffer from. */
     private final PartitionedFileReader fileReader;
 
-    /** Number of remaining non-event buffers to read. */
+    /** Number of remaining non-event buffers in the buffer queue. */
+    @GuardedBy("lock")
     private int dataBufferBacklog;
 
     /** Whether this reader is released or not. */
+    @GuardedBy("lock")
     private boolean isReleased;
 
+    /** Cause of failure which should be propagated to the consumer. */
+    @GuardedBy("lock")
+    private Throwable failureCause;
+
     /** Sequence number of the next buffer to be sent to the consumer. */
     private int sequenceNumber;
 
     public SortMergeSubpartitionReader(
-            int subpartitionIndex,
-            int dataBufferBacklog,
-            int bufferSize,
             SortMergeResultPartition partition,
             BufferAvailabilityListener listener,
-            PartitionedFile partitionedFile)
-            throws IOException {
+            PartitionedFileReader fileReader) {
         this.partition = checkNotNull(partition);
         this.availabilityListener = checkNotNull(listener);
-        this.dataBufferBacklog = dataBufferBacklog;
-
-        // allocate two pieces of unmanaged segments for data reading
-        for (int i = 0; i < NUM_READ_BUFFERS; i++) {
-            this.readBuffers.add(
-                    
MemorySegmentFactory.allocateUnpooledOffHeapMemory(bufferSize, null));
-        }
-
-        this.fileReader = new PartitionedFileReader(partitionedFile, 
subpartitionIndex);
-        try {
-            readBuffers();
-        } catch (Throwable throwable) {
-            // ensure that the file reader is closed when any exception occurs
-            IOUtils.closeQuietly(fileReader);
-            throw throwable;
-        }
+        this.fileReader = checkNotNull(fileReader);
     }
 
     @Nullable
     @Override
     public BufferAndBacklog getNextBuffer() {
-        checkState(!isReleased, "Reader is already released.");
+        synchronized (lock) {
+            Buffer buffer = buffersRead.poll();
+            if (buffer == null) {
+                return null;
+            }
+
+            if (buffer.isBuffer()) {
+                --dataBufferBacklog;
+            }
 
-        Buffer buffer = buffersRead.poll();
-        if (buffer == null) {
-            return null;
+            Buffer lookAhead = buffersRead.peek();
+            return BufferAndBacklog.fromBufferAndLookahead(
+                    buffer,
+                    lookAhead == null ? Buffer.DataType.NONE : 
lookAhead.getDataType(),
+                    dataBufferBacklog,
+                    sequenceNumber++);
         }
+    }
 
-        if (buffer.isBuffer()) {
-            --dataBufferBacklog;
+    private void addBuffer(Buffer buffer) {
+        boolean notifyAvailable;
+        synchronized (lock) {
+            if (isReleased) {
+                buffer.recycleBuffer();
+                return;
+            }
+
+            notifyAvailable = buffersRead.isEmpty();
+
+            buffersRead.add(buffer);
+            if (buffer.isBuffer()) {
+                ++dataBufferBacklog;
+            }
         }
 
-        final Buffer lookAhead = buffersRead.peek();
+        if (notifyAvailable) {
+            notifyDataAvailable();
+        }
+    }
+
+    boolean readBuffers(Queue<MemorySegment> buffers, BufferRecycler recycler) 
throws IOException {
+        while (!buffers.isEmpty()) {
+            MemorySegment segment = buffers.poll();
 
-        return BufferAndBacklog.fromBufferAndLookahead(
-                buffer,
-                lookAhead == null ? Buffer.DataType.NONE : 
lookAhead.getDataType(),
-                dataBufferBacklog,
-                sequenceNumber++);
+            Buffer buffer;
+            try {
+                if ((buffer = fileReader.readCurrentRegion(segment, recycler)) 
== null) {
+                    buffers.add(segment);
+                    break;
+                }
+            } catch (Throwable throwable) {
+                buffers.add(segment);
+                throw throwable;
+            }
+            addBuffer(buffer);
+        }
+        return fileReader.hasRemaining();
     }
 
-    void readBuffers() throws IOException {
-        // we do not need to recycle the allocated segment here if any 
exception occurs
-        // for this subpartition reader will be released so no resource will 
be leaked
-        MemorySegment segment;
-        while ((segment = readBuffers.poll()) != null) {
-            Buffer buffer = fileReader.readBuffer(segment, this);
-            if (buffer == null) {
-                readBuffers.add(segment);
-                break;
+    void fail(Throwable throwable) {
+        checkArgument(throwable != null, "Must be not null.");
+
+        synchronized (lock) {
+            if (isReleased) {
+                return;
+            }
+
+            if (failureCause == null) {
+                failureCause = throwable;
             }
-            buffersRead.add(buffer);
         }
+
+        releaseAllResources();

Review comment:
       Could we extract a new `releaseResourceUnsafe(@Nullable Throwable 
throawable)` method and call it in both `fail()` and `releaseAllResource()` 
method ? Then we do not need to run duplicate `if (isReleased) {..}`

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReader.java
##########
@@ -176,21 +202,28 @@ public void resumeConsumption() {
 
     @Override
     public Throwable getFailureCause() {
-        // we can never throw an error after this was created
-        return null;
+        synchronized (lock) {
+            return failureCause;
+        }
     }
 
     @Override
     public boolean isAvailable(int numCreditsAvailable) {
-        if (numCreditsAvailable > 0) {
-            return !buffersRead.isEmpty();
-        }
+        synchronized (lock) {
+            if (isReleased) {
+                return true;
+            }
 
-        return !buffersRead.isEmpty() && !buffersRead.peek().isBuffer();
+            if (numCreditsAvailable > 0) {

Review comment:
       Perhaps change to 
   
   ```
   return !buffersRead.isEmpty() && (numCredit > 0 || 
!bufferReader.peek().isBuffer())
   ```
   
   or 
   
   ```
   
   if (buffersRead.isEmpty()) {
         return false;
   }
   
   return numCreditsAvailable > 0 || !buffersRead.peek().isBuffer();
   ```
   
   to avoid duplicate condition ?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReader.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.partition;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
+import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+import org.apache.flink.util.IOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Data reader for {@link SortMergeResultPartition} which can read data for 
all downstream tasks
+ * consuming the corresponding {@link SortMergeResultPartition}. It always 
tries to read shuffle
+ * data in order of file offset, which maximums the sequential read so can 
improve the blocking
+ * shuffle performance.
+ */
+public class SortMergeResultPartitionReader implements Runnable, 
BufferRecycler {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(SortMergeResultPartitionReader.class);
+
+    /** Lock used to synchronize multi-thread access to thread-unsafe fields. 
*/
+    private final Object lock;
+
+    /** Buffer pool from which to allocate buffers for shuffle data reading. */
+    private final BatchShuffleReadBufferPool bufferPool;
+
+    /** Executor to run the shuffle data reading task. */
+    private final Executor ioExecutor;
+
+    /** Maximum number of buffers can be allocated by this partition reader. */
+    private final int maxRequestedBuffers;
+
+    /** All failed subpartition readers to be released. */
+    @GuardedBy("lock")
+    private final Set<SortMergeSubpartitionReader> failedReaders = new 
HashSet<>();
+
+    /** All readers waiting to read data of different subpartitions. */
+    @GuardedBy("lock")
+    private final Set<SortMergeSubpartitionReader> allReaders = new 
HashSet<>();
+
+    /** File channel shared by all subpartitions to read data from. */
+    @GuardedBy("lock")
+    private FileChannel dataFileChannel;
+
+    /** File channel shared by all subpartitions to read index from. */
+    @GuardedBy("lock")
+    private FileChannel indexFileChannel;
+
+    /** Number of buffers already allocated and still not recycled by this 
partition reader. */
+    @GuardedBy("lock")
+    private int numRequestedBuffers;
+
+    /**
+     * Whether the data reading task is currently running or not. This flag is 
used when trying to
+     * submit the data reading task.
+     */
+    @GuardedBy("lock")
+    private boolean isRunning;
+
+    /** Whether this reader has been released or not. */
+    @GuardedBy("lock")
+    private boolean isReleased;
+
+    public SortMergeResultPartitionReader(
+            BatchShuffleReadBufferPool bufferPool, Executor ioExecutor, Object 
lock) {
+        this.lock = checkNotNull(lock);
+        this.bufferPool = checkNotNull(bufferPool);
+        this.ioExecutor = checkNotNull(ioExecutor);
+        // one partition reader can consume at most 32M buffers for data read, 
it is only an
+        // empirical value which can not be configured currently
+        this.maxRequestedBuffers = Math.max(1, 4 * 
bufferPool.getNumBuffersPerRequest());
+
+        bufferPool.initialize();
+    }
+
+    @Override
+    public synchronized void run() {
+        Set<SortMergeSubpartitionReader> finishedReaders = new HashSet<>();
+        Queue<SortMergeSubpartitionReader> availableReaders = 
getAvailableReaders();
+
+        Queue<MemorySegment> buffers = new ArrayDeque<>();
+        if (!availableReaders.isEmpty()) {
+            try {
+                buffers.addAll(bufferPool.requestBuffers());
+            } catch (Throwable throwable) {
+                // this can happen only when the buffer pool is destroyed or 
buffer request timeout
+                if (!(throwable instanceof TimeoutException) || 
numRequestedBuffers <= 0) {
+                    failSubpartitionReaders(availableReaders, throwable);
+                    LOG.error("Failed to request buffers for data reading.", 
throwable);

Review comment:
       It seems we would still need to do some cleanup at last, but I still 
tend to extract the cleanup into separate method and return directly here.
   
   Perhaps as a whole we could change the logics here to be 
   
   ```
   xx = getAvailableReaders();
   if (xx.isEmpty()) {
       return;
   }
   
   // allocates buffers
   try {
        xxx
   } catch (Exception e) {
        recycleBuffers(buffers);
        return;
   }
   
   .....
   recycleBuffers();
   removeFinishedAndFailedReaders(xx)
   
   ....
   
   
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to