You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/06/13 18:19:17 UTC

[GitHub] [pulsar] BewareMyPower opened a new pull request, #16043: [improve][broker] Avoid reconnection when a partitioned topic was cre…

BewareMyPower opened a new pull request, #16043:
URL: https://github.com/apache/pulsar/pull/16043

   …ated concurrently
   
   ### Motivation
   
   When a partitioned topic was created concurrently, especially when
   automatically created by many producers. This case can be reproduced
   easily by configuring `allowAutoTopicCreationType=non-partitioned` and
   starting a Pulsar standalone. Then, run the following code:
   
   ```java
   try (PulsarClient client = PulsarClient.builder()
           .serviceUrl("pulsar://localhost:6650").build()) {
       for (int i = 0; i < 10; i++) {
           client.newProducer().topic("topic").createAsync();
       }
       Thread.sleep(1000);
   }
   ```
   
   We can see a lot of "Could not get connection while
   getPartitionedTopicMetadata" warning logs at client side, while there
   were more warning logs with full stack traces at broker side:
   
   ```
   2022-06-14T02:04:20,522+0800 [metadata-store-22-1] WARN  org.apache.pulsar.broker.service.ServerCnx - Failed to get Partitioned Metadata [/127.0.0.1:64846] persistent://public/default/topic: org.apache.pulsar.metadata.api.MetadataStoreException$BadVersionException: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /admin/partitioned-topics/public/default/persistent/topic
   org.apache.pulsar.metadata.api.MetadataStoreException$AlreadyExistsException: org.apache.pulsar.metadata.api.MetadataStoreException$BadVersionException: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /admin/partitioned-topics/public/default/persistent/topic
   ```
   
   It's because when broker handles the partitioned metadata command, it
   calls `fetchPartitionedTopicMetadataCheckAllowAutoCreationAsync` and
   will try creating a partitioned topic if it doesn't exist. It's a race
   condition that if many connections are established during a short time
   interval and one of them created successfully, the following will fail
   with the `AlreadyExistsException`.
   
   ### Modifications
   
   Handles the `MetadataStoreException.AlreadyExistsException` in
   `unsafeGetPartitionedTopicMetadataAsync`. In this case, invoke
   `fetchPartitionedTopicMetadataAsync` to get the partitioned metadata
   again.
   
   ### Verifying this change
   
   Even if without this patch, the creation of producers could also succeed
   because they will reconnect to broker again after 100 ms because broker
   will return a `ServiceNotReady` error in thiss case. The only way to
   verify this fix is reproducing the bug again with this patch, we can
   see no reconnection will happen from the logs.
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [x] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower commented on pull request #16043: [improve][broker] Avoid reconnection when a partitioned topic was created concurrently

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on PR #16043:
URL: https://github.com/apache/pulsar/pull/16043#issuecomment-1199002372

   Move the `release/2.8.4` label to https://github.com/apache/pulsar/pull/16867


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] dave2wave commented on pull request #16043: [improve][broker] Avoid reconnection when a partitioned topic was created concurrently

Posted by GitBox <gi...@apache.org>.
dave2wave commented on PR #16043:
URL: https://github.com/apache/pulsar/pull/16043#issuecomment-1154353030

   To me it would help if this PR were tested using OMB which creates many topic-partitions at once from several clients.


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower merged pull request #16043: [improve][broker] Avoid reconnection when a partitioned topic was created concurrently

Posted by GitBox <gi...@apache.org>.
BewareMyPower merged PR #16043:
URL: https://github.com/apache/pulsar/pull/16043


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] BewareMyPower commented on a diff in pull request #16043: [improve][broker] Avoid reconnection when a partitioned topic was created concurrently

Posted by GitBox <gi...@apache.org>.
BewareMyPower commented on code in PR #16043:
URL: https://github.com/apache/pulsar/pull/16043#discussion_r896341600


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -4085,9 +4085,21 @@ public static CompletableFuture<PartitionedTopicMetadata> unsafeGetPartitionedTo
                 }
                 metadataFuture.complete(metadata);
             }).exceptionally(ex -> {
-            metadataFuture.completeExceptionally(ex.getCause());
-            return null;
-        });
+                    if (ex.getCause() instanceof MetadataStoreException.AlreadyExistsException) {
+                        // The partitioned topic might be created concurrently
+                        pulsar.getBrokerService().fetchPartitionedTopicMetadataAsync(topicName)

Review Comment:
   Good suggestions. I was concerned about the effect on other places before. But it should be the right behavior for it, including the admin operations.



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] RobertIndie commented on a diff in pull request #16043: [improve][broker] Avoid reconnection when a partitioned topic was created concurrently

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on code in PR #16043:
URL: https://github.com/apache/pulsar/pull/16043#discussion_r896298451


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -4085,9 +4085,21 @@ public static CompletableFuture<PartitionedTopicMetadata> unsafeGetPartitionedTo
                 }
                 metadataFuture.complete(metadata);
             }).exceptionally(ex -> {
-            metadataFuture.completeExceptionally(ex.getCause());
-            return null;
-        });
+                    if (ex.getCause() instanceof MetadataStoreException.AlreadyExistsException) {
+                        // The partitioned topic might be created concurrently
+                        pulsar.getBrokerService().fetchPartitionedTopicMetadataAsync(topicName)

Review Comment:
   Why don't we move this exception capture to here: https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L2642-L2645 
   



-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org