codelipenghui commented on code in PR #20718:
URL: https://github.com/apache/pulsar/pull/20718#discussion_r1260422847


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -2070,64 +2070,70 @@ private void getLargestBatchIndexWhenPossible(
 
         // If it's not pointing to a valid entry, respond messageId of the 
current position.
         // If the compaction cursor reach the end of the topic, respond 
messageId from compacted ledger
-        Optional<Position> compactionHorizon = 
persistentTopic.getCompactedTopic().getCompactionHorizon();
-        if (lastPosition.getEntryId() == -1 || (compactionHorizon.isPresent()
-                        && lastPosition.compareTo((PositionImpl) 
compactionHorizon.get()) <= 0)) {
-            handleLastMessageIdFromCompactedLedger(persistentTopic, requestId, 
partitionIndex,
-                    markDeletePosition);
-            return;
-        }
-
-        // For a valid position, we read the entry out and parse the batch 
size from its metadata.
-        CompletableFuture<Entry> entryFuture = new CompletableFuture<>();
-        ml.asyncReadEntry(lastPosition, new AsyncCallbacks.ReadEntryCallback() 
{
-            @Override
-            public void readEntryComplete(Entry entry, Object ctx) {
-                entryFuture.complete(entry);
+        CompletableFuture<Position> compactionHorizonFuture =
+                
persistentTopic.getTopicCompactionService().getLastCompactedPosition();
+
+        compactionHorizonFuture.thenAccept(compactionHorizon -> {
+            if (lastPosition.getEntryId() == -1 || (compactionHorizon != null
+                    && lastPosition.compareTo((PositionImpl) 
compactionHorizon) <= 0)) {
+                handleLastMessageIdFromCompactionService(persistentTopic, 
requestId, partitionIndex,
+                        markDeletePosition);
+                return;
             }
 
-            @Override
-            public void readEntryFailed(ManagedLedgerException exception, 
Object ctx) {
-                entryFuture.completeExceptionally(exception);
-            }
-        }, null);
+            // For a valid position, we read the entry out and parse the batch 
size from its metadata.
+            CompletableFuture<Entry> entryFuture = new CompletableFuture<>();
+            ml.asyncReadEntry(lastPosition, new 
AsyncCallbacks.ReadEntryCallback() {
+                @Override
+                public void readEntryComplete(Entry entry, Object ctx) {
+                    entryFuture.complete(entry);
+                }
 
-        CompletableFuture<Integer> batchSizeFuture = 
entryFuture.thenApply(entry -> {
-            MessageMetadata metadata = 
Commands.parseMessageMetadata(entry.getDataBuffer());
-            int batchSize = metadata.getNumMessagesInBatch();
-            entry.release();
-            return metadata.hasNumMessagesInBatch() ? batchSize : -1;
-        });
+                @Override
+                public void readEntryFailed(ManagedLedgerException exception, 
Object ctx) {
+                    entryFuture.completeExceptionally(exception);
+                }
+            }, null);
+
+            CompletableFuture<Integer> batchSizeFuture = 
entryFuture.thenApply(entry -> {
+                MessageMetadata metadata = 
Commands.parseMessageMetadata(entry.getDataBuffer());
+                int batchSize = metadata.getNumMessagesInBatch();
+                entry.release();
+                return metadata.hasNumMessagesInBatch() ? batchSize : -1;
+            });
 
-        batchSizeFuture.whenComplete((batchSize, e) -> {
-            if (e != null) {
-                if (e.getCause() instanceof 
ManagedLedgerException.NonRecoverableLedgerException) {
-                    handleLastMessageIdFromCompactedLedger(persistentTopic, 
requestId, partitionIndex,
-                            markDeletePosition);
+            batchSizeFuture.whenComplete((batchSize, e) -> {
+                if (e != null) {
+                    if (e.getCause() instanceof 
ManagedLedgerException.NonRecoverableLedgerException) {
+                        
handleLastMessageIdFromCompactionService(persistentTopic, requestId, 
partitionIndex,
+                                markDeletePosition);
+                    } else {
+                        writeAndFlush(Commands.newError(
+                                requestId, ServerError.MetadataError,
+                                "Failed to get batch size for entry " + 
e.getMessage()));
+                    }
                 } else {
-                    writeAndFlush(Commands.newError(
-                            requestId, ServerError.MetadataError,
-                            "Failed to get batch size for entry " + 
e.getMessage()));
-                }
-            } else {
-                int largestBatchIndex = batchSize > 0 ? batchSize - 1 : -1;
+                    int largestBatchIndex = batchSize > 0 ? batchSize - 1 : -1;
 
-                if (log.isDebugEnabled()) {
-                    log.debug("[{}] [{}][{}] Get LastMessageId {} 
partitionIndex {}", remoteAddress,
-                            topic.getName(), subscriptionName, lastPosition, 
partitionIndex);
-                }
+                    if (log.isDebugEnabled()) {
+                        log.debug("[{}] [{}][{}] Get LastMessageId {} 
partitionIndex {}", remoteAddress,
+                                topic.getName(), subscriptionName, 
lastPosition, partitionIndex);
+                    }
 
-                writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, 
lastPosition.getLedgerId(),
-                        lastPosition.getEntryId(), partitionIndex, 
largestBatchIndex,
-                        markDeletePosition != null ? 
markDeletePosition.getLedgerId() : -1,
-                        markDeletePosition != null ? 
markDeletePosition.getEntryId() : -1));
-            }
+                    
writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, 
lastPosition.getLedgerId(),
+                            lastPosition.getEntryId(), partitionIndex, 
largestBatchIndex,
+                            markDeletePosition != null ? 
markDeletePosition.getLedgerId() : -1,
+                            markDeletePosition != null ? 
markDeletePosition.getEntryId() : -1));
+                }
+            });
+        }).exceptionally(e -> {
+            writeAndFlush(Commands.newError(requestId, 
ServerError.UnknownError, e.getMessage()));

Review Comment:
   Is it better to use `MetadataError`? I mean the `compactionHorizon` is more 
like metadata of the topic.
   



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