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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java:
##########
@@ -188,14 +222,34 @@ protected CompletableFuture<Boolean> isLocalTopicActive() 
{
         }, brokerService.executor());
     }
 
+    public synchronized CompletableFuture<Void> disconnect(boolean 
failIfHasBacklog) {
+        if (failIfHasBacklog && getNumberOfEntriesInBacklog() > 0) {
+            CompletableFuture<Void> disconnectFuture = new 
CompletableFuture<>();
+            disconnectFuture.completeExceptionally(new 
TopicBusyException("Cannot close a replicator with backlog"));
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Replicator disconnect failed since topic has 
backlog", replicatorId);
+            }
+            return disconnectFuture;
+        }
+
+        log.info("[{}] Disconnect replicator at position {} with backlog {}", 
replicatorId,
+                getReplicatorReadPosition(), getNumberOfEntriesInBacklog());
+        if (!tryChangeStatusToTerminating()) {

Review Comment:
   It should be `tryChangeStatusToStopping()`?
   
   And it's better add a log here.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java:
##########
@@ -163,16 +175,38 @@ public synchronized void startProducer() {
 
     }
 
-    protected void checkTopicActiveAndRetryStartProducer() {
-        isLocalTopicActive().thenAccept(isTopicActive -> {
-            if (isTopicActive) {
-                startProducer();
+    protected void scheduleCheckTopicActiveAndStartProducer(final long 
waitTimeMs) {

Review Comment:
   We'd better change the log at line:153
   
   ```
   log.info("[{}] Skip the producer creation since the replicator state is : 
{}", replicatorId, state);
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java:
##########
@@ -163,16 +175,38 @@ public synchronized void startProducer() {
 
     }
 
-    protected void checkTopicActiveAndRetryStartProducer() {
-        isLocalTopicActive().thenAccept(isTopicActive -> {
-            if (isTopicActive) {
-                startProducer();
+    protected void scheduleCheckTopicActiveAndStartProducer(final long 
waitTimeMs) {
+        brokerService.executor().schedule(() -> {
+            if (state == State.Terminated) {

Review Comment:
   And it looks like we also have 4 more case below to skip the producer 
creation. Is it better to have a log for each of them? We can have a method to 
print the log with the skipped reason.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java:
##########
@@ -211,35 +265,67 @@ protected synchronized CompletableFuture<Void> 
closeProducerAsync() {
         });
     }
 
-
-    public CompletableFuture<Void> disconnect() {
-        return disconnect(false);
+    public synchronized CompletableFuture<Void> terminate() {
+        if (producer == null) {

Review Comment:
   Do we need to add `tryChangeStatusToTerminating();` before this check?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java:
##########
@@ -163,16 +175,38 @@ public synchronized void startProducer() {
 
     }
 
-    protected void checkTopicActiveAndRetryStartProducer() {
-        isLocalTopicActive().thenAccept(isTopicActive -> {
-            if (isTopicActive) {
-                startProducer();
+    protected void scheduleCheckTopicActiveAndStartProducer(final long 
waitTimeMs) {
+        brokerService.executor().schedule(() -> {
+            if (state == State.Terminated) {
+                return;
             }
-        }).exceptionally(ex -> {
-            log.warn("[{}] Stop retry to create producer due to topic load 
fail. Replicator state: {}", replicatorId,
-                    STATE_UPDATER.get(this), ex);
-            return null;
-        });
+            CompletableFuture<Optional<Topic>> topicFuture = 
brokerService.getTopics().get(localTopicName);
+            if (topicFuture == null) {
+                // Topic closed.
+                return;
+            }
+            topicFuture.thenAccept(optional -> {
+                if (optional.isEmpty()) {
+                    // Topic closed.
+                    return;
+                }
+                if (optional.get() != localTopic) {
+                    // Topic closed and created a new one, current replicator 
is outdated.
+                    return;
+                }
+                // TODO check isClosing or Deleting.

Review Comment:
   Do we need to keep this TODO? It looks like we already checked the state in 
the `startProducer();` method.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java:
##########
@@ -163,16 +175,38 @@ public synchronized void startProducer() {
 
     }
 
-    protected void checkTopicActiveAndRetryStartProducer() {
-        isLocalTopicActive().thenAccept(isTopicActive -> {
-            if (isTopicActive) {
-                startProducer();
+    protected void scheduleCheckTopicActiveAndStartProducer(final long 
waitTimeMs) {
+        brokerService.executor().schedule(() -> {
+            if (state == State.Terminated) {

Review Comment:
   ```suggestion
                if (state == State.Terminating || state == State.Terminated) {
   ```
   
   And it's better to add an info level log here so that we can know why the 
producer creation is skipped.



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