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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7a066f9  RATIS-1495. When DataStreamManagement#read an exception 
occurs, release ByteBuf (#585)
7a066f9 is described below

commit 7a066f9c8e55542b90a06746cb1c01ad4b998e36
Author: hao guo <[email protected]>
AuthorDate: Sat Jan 22 13:05:47 2022 +0800

    RATIS-1495. When DataStreamManagement#read an exception occurs, release 
ByteBuf (#585)
---
 .../org/apache/ratis/netty/server/DataStreamManagement.java    | 10 ++++++++++
 1 file changed, 10 insertions(+)

diff --git 
a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java
 
b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java
index a817372..31634bb 100644
--- 
a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java
+++ 
b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java
@@ -385,6 +385,16 @@ public class DataStreamManagement {
       CheckedBiFunction<RaftClientRequest, Set<RaftPeer>, 
Set<DataStreamOutputRpc>, IOException> getStreams) {
     LOG.debug("{}: read {}", this, request);
     final ByteBuf buf = request.slice();
+    try {
+      readImpl(request, ctx, buf, getStreams);
+    } catch (Throwable t) {
+      buf.release();
+      throw t;
+    }
+  }
+
+  private void readImpl(DataStreamRequestByteBuf request, 
ChannelHandlerContext ctx, ByteBuf buf,
+      CheckedBiFunction<RaftClientRequest, Set<RaftPeer>, 
Set<DataStreamOutputRpc>, IOException> getStreams) {
     boolean close = WriteOption.containsOption(request.getWriteOptions(), 
StandardWriteOption.CLOSE);
     ClientInvocationId key =  
ClientInvocationId.valueOf(request.getClientId(), request.getStreamId());
     final StreamInfo info;

Reply via email to