You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/06/22 08:19:12 UTC

[GitHub] [pulsar] aloyszhang opened a new pull request #11014: fix deserialized error

aloyszhang opened a new pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014


   
   
   Fixes #11013
   
   ### Motivation
   
   Fix deserialized error caused by not check field before get.
   
   
   ### Modifications
   
   Check `publish_time` exist before get this field.
   
   ### Verifying this change
   
   
   This change is already covered by existing tests, such as *(please describe tests)*.
   
   
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: ( no)
     - The schema: ( no )
     - The default values of configurations: ( no)
     - The wire protocol: ( no)
     - The rest endpoints: ( no)
     - The admin cli options: ( no)
     - Anything that affects deployment: (no)
   
   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-869567183


   /pulsarbot run-failure-checks


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r663089943



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageImplTest.java
##########
@@ -450,10 +450,8 @@ public void testMessageBrokerAndEntryMetadataTimestampMissed() {
 
             CompositeByteBuf compositeByteBuf = PulsarByteBufAllocator.DEFAULT.compositeBuffer();
             compositeByteBuf.addComponents(true, brokerMeta, byteBuf);
-            MessageImpl messageWithEntryMetadata = MessageImpl.deserializeBrokerEntryMetaDataFirst(compositeByteBuf);
-            MessageImpl message = MessageImpl.deserializeSkipBrokerEntryMetaData(compositeByteBuf);
-            message.setBrokerEntryMetadata(messageWithEntryMetadata.getBrokerEntryMetadata());
-            assertTrue(message.isExpired(100));
+            long entryTimestamp = MessageImpl.getEntryTimestamp(compositeByteBuf);

Review comment:
       fixed

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,31 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
+        }
+        // otherwise get the publish_time,
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
+        try {
+            Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
+            return msg.getPublishTime();
+        } finally {
+            // make sure msg can be recycled
+            msg.recycle();
         }
+    }
 
-        Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+    public static boolean isEntryExpired(int messageTTLInSeconds, long entryTimestamp) {
+        return messageTTLInSeconds != 0 &&
+                (System.currentTimeMillis() > entryTimestamp + TimeUnit.SECONDS.toMillis(messageTTLInSeconds));
+    }
+
+    public static boolean isEntryPublishedEarlierThan(long timestamp, long entryTimestamp) {

Review comment:
       fixed




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r658126131



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageImplTest.java
##########
@@ -425,7 +425,7 @@ public void testTypedSchemaGetNullValue() {
         assertNull(msg.getValue());
     }
 
-    @Test(timeOut = 30000)
+    @Test(timeOut = 300000)

Review comment:
       Increase this for debugging.  I'll reset to 30s.

##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageImplTest.java
##########
@@ -425,7 +425,7 @@ public void testTypedSchemaGetNullValue() {
         assertNull(msg.getValue());
     }
 
-    @Test(timeOut = 30000)
+    @Test(timeOut = 300000)

Review comment:
       Increase this for debugging.  I'll reset it to 30s.




-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-873170828


   @hangc0276 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] hangc0276 commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r663302514



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,26 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
+        // otherwise get the publish_time,
+        MessageMetadata messageMetadata = new MessageMetadata();

Review comment:
       MessageMetadata messageMetadata = LOCAL_MESSAGE_METADATA.get();
   maybe better.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659524869



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,26 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();

Review comment:
       There's a memory leak here.
   
   Before the change, the `MessageImpl<byte[]>` returned by `deserializeBrokerEntryMetaDataFirst` before will call `recycle()` method in caller side like `expireMessages`, `findMessages`, etc. 
   
   After the change, `msg` here never calls `recycle()` method.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang edited a comment on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang edited a comment on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-873170828


   @hangc0276  Thanks for your comments, I've resolved them. 
   PTAL again.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower merged pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower merged pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-867972963


   /pulsarbot run-failure-checks


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659664596



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,28 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
         Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+        long entryTimestamp = msg.getPublishTime();
+        msg.recycle();
+        return entryTimestamp;

Review comment:
       Do you mean `msg.getPublishTime()` may throws `IllegalArgumentException` if the `publish_time` is not set ?
   Since `publish_time` is a  required field,  there must be something wroing if it's not set. 
   Using try-catch-finally will hide the reason, so maybe, I think it's better to throw the exceptin.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-870230975


   cc @hangc0276 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r663307456



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,26 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
+        // otherwise get the publish_time,
+        MessageMetadata messageMetadata = new MessageMetadata();

Review comment:
       fixed




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868353422


   @BewareMyPower @hangc0276 PTAL


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r658126131



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageImplTest.java
##########
@@ -425,7 +425,7 @@ public void testTypedSchemaGetNullValue() {
         assertNull(msg.getValue());
     }
 
-    @Test(timeOut = 30000)
+    @Test(timeOut = 300000)

Review comment:
       Increase this for debugging.  Reset to 30s.




-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-873498280


   @hangc0276 PTAL


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868046884


   /pulsarbot run-failure-checks


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868180347


   @lhotari Hi could you help look at this test failure?
   
   ```
   Error:  Tests run: 14, Failures: 1, Errors: 0, Skipped: 13, Time elapsed: 57.774 s <<< FAILURE! - in org.apache.pulsar.broker.resourcegroup.ResourceGroupConfigListenerTest
   Error:  testResourceGroupAttachToNamespace(org.apache.pulsar.broker.resourcegroup.ResourceGroupConfigListenerTest)  Time elapsed: 10.167 s  <<< FAILURE!
   org.awaitility.core.ConditionTimeoutException: Assertion condition defined as a lambda expression in org.apache.pulsar.broker.resourcegroup.ResourceGroupConfigListenerTest expected object to not be null within 10 seconds.
   ```
   
   I see your PR (https://github.com/apache/pulsar/pull/11048) fixed the flaky test, but it still failed for multiple times.


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] hangc0276 commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r662997583



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,26 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();

Review comment:
       Why not just init a `MessageMetadata` object instead of `MessageImpl` ?

##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageImplTest.java
##########
@@ -450,10 +450,8 @@ public void testMessageBrokerAndEntryMetadataTimestampMissed() {
 
             CompositeByteBuf compositeByteBuf = PulsarByteBufAllocator.DEFAULT.compositeBuffer();
             compositeByteBuf.addComponents(true, brokerMeta, byteBuf);
-            MessageImpl messageWithEntryMetadata = MessageImpl.deserializeBrokerEntryMetaDataFirst(compositeByteBuf);
-            MessageImpl message = MessageImpl.deserializeSkipBrokerEntryMetaData(compositeByteBuf);
-            message.setBrokerEntryMetadata(messageWithEntryMetadata.getBrokerEntryMetadata());
-            assertTrue(message.isExpired(100));
+            long entryTimestamp = MessageImpl.getEntryTimestamp(compositeByteBuf);

Review comment:
       Please add an assert to check `entryTimestamp` value

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,31 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
+        }
+        // otherwise get the publish_time,
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
+        try {
+            Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
+            return msg.getPublishTime();
+        } finally {
+            // make sure msg can be recycled
+            msg.recycle();
         }
+    }
 
-        Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+    public static boolean isEntryExpired(int messageTTLInSeconds, long entryTimestamp) {
+        return messageTTLInSeconds != 0 &&
+                (System.currentTimeMillis() > entryTimestamp + TimeUnit.SECONDS.toMillis(messageTTLInSeconds));
+    }
+
+    public static boolean isEntryPublishedEarlierThan(long timestamp, long entryTimestamp) {

Review comment:
       we'd better replace the order of the two parameters according to your method name.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-873152058


   /pulsarbot run-failure-checks


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r658091677



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageImplTest.java
##########
@@ -425,7 +425,7 @@ public void testTypedSchemaGetNullValue() {
         assertNull(msg.getValue());
     }
 
-    @Test(timeOut = 30000)
+    @Test(timeOut = 300000)

Review comment:
       Why did you increase timeout from 30 seconds to 300 seconds?




-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868153825


   /pulsarbot run-failure-checks


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868328129


   /pulsarbot run-failure-checks


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #11014: fix deserialized error

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-865711649


   After discussing with @aloyszhang , we need to find why the **required** `publish_time` field is missed first.


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-869695955


   @hangc0276 Could you also take a look?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-870230975


   cc @hangc0276 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868165158


   /pulsarbot run-failure-checks


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang removed a comment on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang removed a comment on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868153825






-- 
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:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-869695955


   @hangc0276 Could you also take a look?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659664596



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,28 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
         Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+        long entryTimestamp = msg.getPublishTime();
+        msg.recycle();
+        return entryTimestamp;

Review comment:
       Do you mean `msg.getPublishTime()` may throws `IllegalArgumentException` if the `publish_time` is not set ?
   Since `publish_time` is a  required field,  there must be something wroing if it's not set. 
   Using try-catch-finally will hide the reason, so maybe, I think it's better to throws the exceptin.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659664596



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,28 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
         Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+        long entryTimestamp = msg.getPublishTime();
+        msg.recycle();
+        return entryTimestamp;

Review comment:
       Do you mean `msg.getPublishTime()` may throws `IllegalArgumentException` if the `publish_time` is not set ?
   Since `publish_time` is a  required field,  there must be something wroing if it's not set. 
   Using try-catch-finally will hide the reason, so maybe, I think it's better to throw the exceptin.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868153737


   ResourceGroupConfigListenerTest


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang removed a comment on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang removed a comment on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-868153737


   ResourceGroupConfigListenerTest


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-869322961


   @BewareMyPower  I have added a new method named `getEntryTimestamp` which will return the brokerEntryTimestamp if `BrokerEntryMetadata` is enabled or otherwise return the publishTime. And then using this `entryTimestamp` for expiry checking.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659690266



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,28 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
         Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+        long entryTimestamp = msg.getPublishTime();
+        msg.recycle();
+        return entryTimestamp;

Review comment:
       fixed




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-867723742


   /pulsarbot run-failure-checks


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#issuecomment-871573307


   @sijie @codelipenghui PTAL


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] aloyszhang commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
aloyszhang commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659597947



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,26 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();

Review comment:
       You're right!  Fixed.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] BewareMyPower commented on a change in pull request #11014: fix publish_time not set error when broker entry metadata enable without AppendBrokerTimestampMetadataInterceptor

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on a change in pull request #11014:
URL: https://github.com/apache/pulsar/pull/11014#discussion_r659640783



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
##########
@@ -269,31 +269,28 @@ public MessageImpl(String topic, String msgId, Map<String, String> properties,
         return msg;
     }
 
-    public static MessageImpl<byte[]> deserializeBrokerEntryMetaDataFirst(
-            ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
-        @SuppressWarnings("unchecked")
-        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
-
-        msg.brokerEntryMetadata =
+    public static long getEntryTimestamp( ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException {
+        // get broker timestamp first if BrokerEntryMetadata is enabled with AppendBrokerTimestampMetadataInterceptor
+        BrokerEntryMetadata brokerEntryMetadata =
                 Commands.parseBrokerEntryMetadataIfExist(headersAndPayloadWithBrokerEntryMetadata);
-
-        if (msg.brokerEntryMetadata != null) {
-            msg.msgMetadata.clear();
-            msg.payload = null;
-            msg.messageId = null;
-            msg.topic = null;
-            msg.cnx = null;
-            msg.properties = Collections.emptyMap();
-            return msg;
+        if (brokerEntryMetadata != null && brokerEntryMetadata.hasBrokerTimestamp()) {
+            return brokerEntryMetadata.getBrokerTimestamp();
         }
-
+        // otherwise get the publish_time
+        MessageImpl<byte[]> msg = (MessageImpl<byte[]>) RECYCLER.get();
         Commands.parseMessageMetadata(headersAndPayloadWithBrokerEntryMetadata, msg.msgMetadata);
-        msg.payload = headersAndPayloadWithBrokerEntryMetadata;
-        msg.messageId = null;
-        msg.topic = null;
-        msg.cnx = null;
-        msg.properties = Collections.emptyMap();
-        return msg;
+        long entryTimestamp = msg.getPublishTime();
+        msg.recycle();
+        return entryTimestamp;

Review comment:
       It's better to use try-catch-finally block here like the previous code. `parseMessageMetadata` may throw `IllegalArgumentException` from LightProto though it's an unchecked exception.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org