merlimat closed pull request #2316: Fixed refcounting when processing entries 
from sql
URL: https://github.com/apache/incubator-pulsar/pull/2316
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageParser.java 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageParser.java
index 8cf0328613..b95b22dd14 100644
--- 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageParser.java
+++ 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageParser.java
@@ -95,15 +95,10 @@ public static void parseMessage(TopicName topicName, long 
ledgerId, long entryId
             if (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch()) {
                 final MessageImpl<?> message = new MessageImpl<>(msgId, 
msgMetadata, uncompressedPayload, null, null);
                 processor.process(msgId, message, uncompressedPayload);
-
-                uncompressedPayload.release();
-
             } else {
                 // handle batch message enqueuing; uncompressed payload has 
all messages in batch
                 receiveIndividualMessagesFromBatch(msgMetadata, 
uncompressedPayload, messageId, null, -1, processor);
-                uncompressedPayload.release();
             }
-
         } finally {
             if (uncompressedPayload != null) {
                 uncompressedPayload.release();
diff --git 
a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
 
b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
index 618b33bd7a..3523228763 100644
--- 
a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
+++ 
b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
@@ -52,7 +52,6 @@
 import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
 
 import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
 import static com.facebook.presto.spi.type.BigintType.BIGINT;
@@ -202,10 +201,9 @@ public boolean advanceNextPosition() {
                 throw new RuntimeException(e);
             }
 
-            newEntries.forEach(new Consumer<Entry>() {
-                @Override
-                public void accept(Entry entry) {
-                    completedBytes += entry.getData().length;
+            newEntries.forEach(entry -> {
+                try {
+                    completedBytes += entry.getDataBuffer().readableBytes();
                     // filter entries that is not part of my split
                     if (((PositionImpl) 
entry.getPosition()).compareTo(pulsarSplit.getEndPosition()) < 0) {
                         try {
@@ -217,9 +215,9 @@ public void accept(Entry entry) {
                             log.error(e, "Failed to parse message from pulsar 
topic %s", topicName.toString());
                             throw new RuntimeException(e);
                         }
-                    } else {
-                        entry.release();
                     }
+                } finally {
+                    entry.release();
                 }
             });
         }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to