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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1478,25 +1489,14 @@ public synchronized ScheduledExecutorService 
getCompactorExecutor() {
         return this.compactorExecutor;
     }
 
-    // only public so mockito can mock it
-    public Compactor newCompactor() throws PulsarServerException {
-        return new TwoPhaseCompactor(this.getConfiguration(),
-                getClient(), getBookKeeperClient(),
-                getCompactorExecutor());
-    }
-
-    public synchronized Compactor getCompactor() throws PulsarServerException {
-        if (this.compactor == null) {
-            this.compactor = newCompactor();
-        }
-        return this.compactor;
-    }
-
     // This method is used for metrics, which is allowed to as null
     // Because it's no operation on the compactor, so let's remove the  
synchronized on this method
     // to avoid unnecessary lock competition.
     public Compactor getNullableCompactor() {
-        return this.compactor;
+        if (this.compactionServiceFactory instanceof 
PulsarCompactionServiceFactory pulsarCompactedServiceFactory) {
+            return pulsarCompactedServiceFactory.getNullableCompactor();
+        }

Review Comment:
   Please add a comment here. Only the pulsar's compaction service provides the 
compaction stats. The compaction service plugin, it should be done by the 
plugin itself to expose the compaction metrics.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java:
##########
@@ -2070,64 +2070,67 @@ 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 -> {

Review Comment:
   It looks like the exception handling for compactionHorizonFuture is missed.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -296,29 +298,11 @@ public PersistentTopic(String topic, ManagedLedger 
ledger, BrokerService brokerS
                 
brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
         registerTopicPolicyListener();
 
-        this.compactedTopic = new 
CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient());
-
-        for (ManagedCursor cursor : ledger.getCursors()) {
-            if (cursor.getName().equals(DEDUPLICATION_CURSOR_NAME)
-                    || cursor.getName().startsWith(replicatorPrefix)) {
-                // This is not a regular subscription, we are going to
-                // ignore it for now and let the message dedup logic to take 
care of it
-            } else {
-                final String subscriptionName = Codec.decode(cursor.getName());
-                subscriptions.put(subscriptionName, 
createPersistentSubscription(subscriptionName, cursor,
-                        
PersistentSubscription.isCursorFromReplicatedSubscription(cursor),
-                        cursor.getCursorProperties()));
-                // subscription-cursor gets activated by default: deactivate 
as there is no active subscription right
-                // now
-                subscriptions.get(subscriptionName).deactivateCursor();
-            }
-        }
         this.messageDeduplication = new 
MessageDeduplication(brokerService.pulsar(), this, ledger);
         if (ledger.getProperties().containsKey(TOPIC_EPOCH_PROPERTY_NAME)) {
             topicEpoch = 
Optional.of(Long.parseLong(ledger.getProperties().get(TOPIC_EPOCH_PROPERTY_NAME)));
         }
 
-        checkReplicatedSubscriptionControllerState();

Review Comment:
   Why we should remove this line?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -1914,4 +1914,13 @@ public void shutdownNow() {
     protected BrokerService newBrokerService(PulsarService pulsar) throws 
Exception {
         return new BrokerService(pulsar, ioEventLoopGroup);
     }
+
+    private CompactionServiceFactory loadCompactionServiceFactory() {
+        String compactionServiceFactoryClassName = 
config.getCompactionServiceFactoryClassName();
+        var compactionServiceFactory =
+                Reflections.createInstance(compactionServiceFactoryClassName, 
CompactionServiceFactory.class,
+                        Thread.currentThread().getContextClassLoader());
+        compactionServiceFactory.initialize(this).join();

Review Comment:
   Yes, I think it makes sense to change to a sync method because the starting 
broker needs to initialize the compaction service before it provides messaging 
service to the client.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -434,6 +419,32 @@ public AtomicLong getPendingWriteOps() {
         return pendingWriteOps;
     }
 
+    private CompletableFuture<Void> newTopicCompactionService() {
+        CompactionServiceFactory compactionServiceFactory = 
brokerService.pulsar().getCompactionServiceFactory();
+        return 
compactionServiceFactory.newTopicCompactionService(topic).thenAccept(topicCompactionService
 -> {
+            PersistentTopic.this.topicCompactionService = 
topicCompactionService;
+        });
+    }

Review Comment:
   Is it better to move this method to PulsarService?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -481,8 +492,10 @@ public CompletableFuture<Void> unloadSubscription(@Nonnull 
String subName) {
 
     private PersistentSubscription createPersistentSubscription(String 
subscriptionName, ManagedCursor cursor,
             boolean replicated, Map<String, String> subscriptionProperties) {
-        Objects.requireNonNull(compactedTopic);
-        if (isCompactionSubscription(subscriptionName)) {
+        Objects.requireNonNull(topicCompactionService);
+        if (isCompactionSubscription(subscriptionName)
+                && topicCompactionService instanceof 
PulsarTopicCompactionService pulsarTopicCompactionService) {
+            CompactedTopicImpl compactedTopic = 
pulsarTopicCompactionService.getCompactedTopic();

Review Comment:
   What about the compaction service plugin? Will it use the same compaction 
cursor for maintaining the compaction status?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -210,9 +212,9 @@ public class PersistentTopic extends AbstractTopic 
implements Topic, AddEntryCal
 
     protected final MessageDeduplication messageDeduplication;
 
-    private static final long COMPACTION_NEVER_RUN = -0xfebecffeL;
+    private static final Long COMPACTION_NEVER_RUN = -0xfebecffeL;
     private CompletableFuture<Long> currentCompaction = 
CompletableFuture.completedFuture(COMPACTION_NEVER_RUN);
-    private final CompactedTopic compactedTopic;
+    private TopicCompactionService topicCompactionService;

Review Comment:
   Should we close the compaction service when closing(deleting) the topic? We 
should provide a close method for the compaction service to make sure the 
opened resources can be closed when the topic is closed.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -296,29 +298,11 @@ public PersistentTopic(String topic, ManagedLedger 
ledger, BrokerService brokerS
                 
brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
         registerTopicPolicyListener();
 
-        this.compactedTopic = new 
CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient());
-
-        for (ManagedCursor cursor : ledger.getCursors()) {
-            if (cursor.getName().equals(DEDUPLICATION_CURSOR_NAME)
-                    || cursor.getName().startsWith(replicatorPrefix)) {
-                // This is not a regular subscription, we are going to
-                // ignore it for now and let the message dedup logic to take 
care of it
-            } else {
-                final String subscriptionName = Codec.decode(cursor.getName());
-                subscriptions.put(subscriptionName, 
createPersistentSubscription(subscriptionName, cursor,
-                        
PersistentSubscription.isCursorFromReplicatedSubscription(cursor),
-                        cursor.getCursorProperties()));
-                // subscription-cursor gets activated by default: deactivate 
as there is no active subscription right
-                // now
-                subscriptions.get(subscriptionName).deactivateCursor();
-            }
-        }
         this.messageDeduplication = new 
MessageDeduplication(brokerService.pulsar(), this, ledger);
         if (ledger.getProperties().containsKey(TOPIC_EPOCH_PROPERTY_NAME)) {
             topicEpoch = 
Optional.of(Long.parseLong(ledger.getProperties().get(TOPIC_EPOCH_PROPERTY_NAME)));
         }
 
-        checkReplicatedSubscriptionControllerState();

Review Comment:
   Oh, it has been moved to the `initialize` stage.



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