gaoran10 commented on code in PR #21682:
URL: https://github.com/apache/pulsar/pull/21682#discussion_r1423637465


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java:
##########
@@ -1509,14 +1508,25 @@ public CompletableFuture<Void> close(
         shadowReplicators.forEach((__, replicator) -> 
futures.add(replicator.disconnect()));
         if (disconnectClients) {
             futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData(
-                    brokerService.getPulsar(), topic).thenAccept(lookupData ->
-                    producers.values().forEach(producer -> 
futures.add(producer.disconnect(lookupData)))
+                brokerService.getPulsar(), topic).thenAccept(lookupData -> {
+                    producers.values().forEach(producer -> 
futures.add(producer.disconnect(lookupData)));
+                    // Topics unloaded due to the ExtensibleLoadManager 
undergo closing twice: first with
+                    // disconnectClients = false, second with 
disconnectClients = true. The check below identifies the
+                    // cases when Topic.close is called outside the scope of 
the ExtensibleLoadManager. In these
+                    // situations, we must pursue the regular 
Subscription.close, as Topic.close is invoked just once.
+                    if (isTransferring()) {

Review Comment:
   Does this have a similar effect with the flag `disconnectClients`?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java:
##########
@@ -835,16 +835,44 @@ protected void handleCloseProducer(CommandCloseProducer 
closeProducer) {
 
     @Override
     protected void handleCloseConsumer(CommandCloseConsumer closeConsumer) {
-        log.info("[{}] Broker notification of Closed consumer: {}", 
remoteAddress, closeConsumer.getConsumerId());
+        log.info("[{}] Broker notification of Closed consumer: {}, 
assignedBrokerUrl: {}, assignedBrokerUrlTls: {}",
+                remoteAddress, closeConsumer.getConsumerId(),
+                closeConsumer.hasAssignedBrokerServiceUrl() ? 
closeConsumer.getAssignedBrokerServiceUrl() : null,
+                closeConsumer.hasAssignedBrokerServiceUrlTls() ? 
closeConsumer.getAssignedBrokerServiceUrlTls() : null);
         final long consumerId = closeConsumer.getConsumerId();
         ConsumerImpl<?> consumer = consumers.remove(consumerId);
         if (consumer != null) {
-            consumer.connectionClosed(this);
+            String brokerServiceUrl = getBrokerServiceUrl(closeConsumer, 
consumer);
+            Optional<URI> hostUri = parseUri(brokerServiceUrl, 
closeConsumer.getRequestId());
+            Optional<Long> initialConnectionDelayMs = hostUri.map(__ -> 0L);
+            consumer.connectionClosed(this, initialConnectionDelayMs, hostUri);
         } else {
             log.warn("Consumer with id {} not found while closing consumer ", 
consumerId);
         }
     }
 
+    private static String getBrokerServiceUrl(CommandCloseConsumer 
closeConsumer, ConsumerImpl<?> consumer) {
+        if (consumer.getClient().getConfiguration().isUseTls()) {
+            if (closeConsumer.hasAssignedBrokerServiceUrlTls()) {
+                return closeConsumer.getAssignedBrokerServiceUrlTls();
+            }
+        } else if (closeConsumer.hasAssignedBrokerServiceUrl()) {
+            return closeConsumer.getAssignedBrokerServiceUrl();
+        }
+        return null;
+    }
+
+    private Optional<URI> parseUri(String url, long requestId) {

Review Comment:
   Could we use this method while handling close producers?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java:
##########
@@ -665,15 +672,9 @@ protected synchronized boolean 
trySendMessagesToConsumers(ReadType readType, Lis
         long totalEntries = 0;
         int avgBatchSizePerMsg = remainingMessages > 0 ? 
Math.max(remainingMessages / entries.size(), 1) : 1;
 
-        int firstAvailableConsumerPermits, currentTotalAvailablePermits;
-        boolean dispatchMessage;
-        while (entriesToDispatch > 0) {
-            firstAvailableConsumerPermits = getFirstAvailableConsumerPermits();
-            currentTotalAvailablePermits = Math.max(totalAvailablePermits, 
firstAvailableConsumerPermits);
-            dispatchMessage = currentTotalAvailablePermits > 0 && 
firstAvailableConsumerPermits > 0;

Review Comment:
   Why need to remove this check?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java:
##########
@@ -64,13 +65,13 @@ default long getNumberOfEntriesDelayed() {
 
     List<Consumer> getConsumers();
 
-    CompletableFuture<Void> close();

Review Comment:
   Could we retain this interface? I'm unsure if this exists in the PIP, maybe 
we could add a deprecated annotation 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.

To unsubscribe, e-mail: [email protected]

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

Reply via email to