wolfstudy commented on code in PR #15628:
URL: https://github.com/apache/pulsar/pull/15628#discussion_r889792717


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java:
##########
@@ -75,6 +75,18 @@ public boolean expireMessages(int messageTTLInSeconds) {
 
             
cursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries,
 entry -> {
                 try {
+                    // When the time of the delayed message is greater than 
the time specified by TTL, we should
+                    // give up checking the TTL of the current delayed 
message, because the time of the delayed
+                    // message has not yet arrived, we cannot delete these 
messages.
+                    long delayTime = 
MessageImpl.getDelayTime(entry.getDataBuffer());

Review Comment:
   Hello @merlimat 
   
   In here, I define a pojo object of `EntryMetadata` that returns both 
`publishTime` and `delayTime` fields in one getEntry. Avoid multiple getEntry 
causing the following errors:
   
   ```
   8:22:40.475 [BookKeeperClientWorker-OrderedExecutor-11-0] ERROR 
org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor - 
[persistent://cmq-251199518/CMQ_QUEUE-perf_queue_cap-8/perf_queue_cap-8-partition-0][251199518_perf_queue_cap-8_consumer]
 Error deserializing message for expiry check
   
   java.lang.IllegalStateException: Some required fields are missing
   
           at 
org.apache.pulsar.common.api.proto.MessageMetadata.checkRequiredFields(MessageMetadata.java:1378)
 ~[org.apache.pulsar-pulsar-common-2.9.2.jar:2.9.2]
   
           at 
org.apache.pulsar.common.api.proto.MessageMetadata.parseFrom(MessageMetadata.java:1373)
 ~[org.apache.pulsar-pulsar-common-2.9.2.jar:2.9.2]
   
           at 
org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:435)
 ~[org.apache.pulsar-pulsar-common-2.9.2.jar:2.9.2]
   
           at 
org.apache.pulsar.common.protocol.Commands.parseMessageMetadata(Commands.java:422)
 ~[org.apache.pulsar-pulsar-common-2.9.2.jar:2.9.2]
   
           at 
org.apache.pulsar.client.impl.MessageImpl.getDelayTime(MessageImpl.java:286) 
~[org.apache.pulsar-pulsar-client-original-2.9.2.jar:2.9.2]
   
           at 
org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor.lambda$expireMessages$0(PersistentMessageExpiryMonitor.java:82)
 ~[org.apache.pulsar-pulsar-broker-2.9.2.jar:2.9.2]
   
           at 
org.apache.bookkeeper.mledger.impl.OpFindNewest.readEntryComplete(OpFindNewest.java:89)
 ~[org.apache.pulsar-managed-ledger-2.9.2.jar:2.9.2]
   
           at 
org.apache.bookkeeper.mledger.impl.EntryCacheImpl.lambda$asyncReadEntry0$0(EntryCacheImpl.java:222)
 ~[org.apache.pulsar-managed-ledger-2.9.2.jar:2.9.2]
   
           at 
java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670) 
[?:1.8.0_322]
   
           at 
java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
 [?:1.8.0_322]
   
           at 
java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:456)
 [?:1.8.0_322]
   
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
[?:1.8.0_322]
   
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[?:1.8.0_322]
   
           at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 [io.netty-netty-common-4.1.76.Final.jar:4.1.76.Final]
   
           at java.lang.Thread.run(Thread.java:750) [?:1.8.0_322]
   
   
   ```



-- 
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: [email protected]

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

Reply via email to