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/10/25 16:52:10 UTC

[GitHub] [pulsar] dlg99 opened a new pull request, #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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

   ### Motivation
   
   Forced delete of partitioned topic with active consumer on the namespace where the topic autocreate is enabled leaves the namespace in the state where one cannot create partitioned topic with the same name (because it exists already) and cannot delete it (because it does not exist at the same time)
   
   https://github.com/apache/pulsar/pull/17308 -> https://github.com/apache/pulsar/pull/17566 -> this rework of 17308
   
   ### Modifications
   
   Prevents autocreation of the topic partitions during deletion. Correctly detects partitioned topic as non-existing if namespace level metadata is deleted. 
   
   ### Verifying this change
   
   This change added tests
   
   PR validating the change: https://github.com/dlg99/pulsar/pull/2
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
   nothing AFAICT
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API: (yes / no)
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### 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` 
   
   Bug fix
     
   - [ ] `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] eolivelli commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -2886,30 +2888,38 @@ public CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadata
                                     if (metadata.partitions == 0
                                             && !topicExists
                                             && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName, policies)
                                             && pulsar.getBrokerService()
                                                             .isDefaultTopicTypePartitioned(topicName, policies)) {
-
-                                        pulsar.getBrokerService()
-                                                .createDefaultPartitionedTopicAsync(topicName, policies)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    if (ex.getCause()
-                                                            instanceof MetadataStoreException.AlreadyExistsException) {
-                                                        // The partitioned topic might be created concurrently
-                                                        fetchPartitionedTopicMetadataAsync(topicName)
-                                                                .whenComplete((metadata2, ex2) -> {
-                                                                    if (ex2 == null) {
-                                                                        future.complete(metadata2);
-                                                                    } else {
-                                                                        future.completeExceptionally(ex2);
-                                                                    }
-                                                                });
-                                                    } else {
-                                                        future.completeExceptionally(ex);
-                                                    }
-                                                    return null;
-                                                });
+                                        isAllowAutoTopicCreationAsync(topicName, policies).thenAccept(allowed -> {

Review Comment:
   @poorbarcode PTAL here https://github.com/apache/pulsar/pull/18518 



-- 
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] poorbarcode commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -2886,30 +2888,38 @@ public CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadata
                                     if (metadata.partitions == 0
                                             && !topicExists
                                             && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName, policies)
                                             && pulsar.getBrokerService()
                                                             .isDefaultTopicTypePartitioned(topicName, policies)) {
-
-                                        pulsar.getBrokerService()
-                                                .createDefaultPartitionedTopicAsync(topicName, policies)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    if (ex.getCause()
-                                                            instanceof MetadataStoreException.AlreadyExistsException) {
-                                                        // The partitioned topic might be created concurrently
-                                                        fetchPartitionedTopicMetadataAsync(topicName)
-                                                                .whenComplete((metadata2, ex2) -> {
-                                                                    if (ex2 == null) {
-                                                                        future.complete(metadata2);
-                                                                    } else {
-                                                                        future.completeExceptionally(ex2);
-                                                                    }
-                                                                });
-                                                    } else {
-                                                        future.completeExceptionally(ex);
-                                                    }
-                                                    return null;
-                                                });
+                                        isAllowAutoTopicCreationAsync(topicName, policies).thenAccept(allowed -> {

Review Comment:
   How do we make sure the cache data is not outdated? E.g:
   
   | `broker-1` | `broker-2`|
   |---|---|
   |  | read meta from cache |
   | delete topic `tp_test` |  |
   | write meta |  |
   |  | create `tp_test-partion-0` |
   |  | cache update |
   |  | create finish. bingo! |
   



##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java:
##########
@@ -295,6 +296,79 @@ public CompletableFuture<Void> clearPartitionedTopicTenantAsync(String tenant) {
             final String partitionedTopicPath = joinPath(PARTITIONED_TOPIC_PATH, tenant);
             return deleteIfExistsAsync(partitionedTopicPath);
         }
+
+        public CompletableFuture<Void> markPartitionedTopicDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                return CompletableFuture.completedFuture(null);
+            }
+            if (log.isDebugEnabled()) {
+                log.debug("markPartitionedTopicDeletedAsync {}", tn);
+            }
+            return updatePartitionedTopicAsync(tn, md -> {
+                md.deleted = true;
+                return md;
+            });
+        }
+
+        public CompletableFuture<Void> unmarkPartitionedTopicDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                return CompletableFuture.completedFuture(null);
+            }
+            if (log.isDebugEnabled()) {
+                log.debug("unmarkPartitionedTopicDeletedAsync {}", tn);
+            }
+            return updatePartitionedTopicAsync(tn, md -> {
+                md.deleted = false;
+                return md;
+            });
+        }
+
+        public CompletableFuture<Boolean> isPartitionedTopicBeingDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                tn = TopicName.get(tn.getPartitionedTopicName());
+            }
+            return getPartitionedTopicMetadataAsync(tn)
+                    .thenApply(mdOpt -> mdOpt.map(partitionedTopicMetadata -> partitionedTopicMetadata.deleted)
+                            .orElse(false));
+        }
+
+        public CompletableFuture<Void> runWithMarkDeleteAsync(TopicName topic,
+                                                              Supplier<CompletableFuture<Void>> supplier) {
+            CompletableFuture<Void> future = new CompletableFuture<>();
+
+            markPartitionedTopicDeletedAsync(topic).whenCompleteAsync((markResult, markExc) -> {
+                final boolean mdFound;
+                if (markExc != null) {
+                    if (markExc.getCause() instanceof MetadataStoreException.NotFoundException) {
+                        mdFound = false;
+                    } else {
+                        log.error("Failed to mark the topic {} as deleted", topic, markExc);
+                        future.completeExceptionally(markExc);
+                        return;
+                    }
+                } else {
+                    mdFound = true;
+                }
+
+                supplier.get().whenComplete((deleteResult, deleteExc) -> {
+                    if (deleteExc != null && mdFound) {
+                        unmarkPartitionedTopicDeletedAsync(topic)

Review Comment:
   If the broker is down at this point, we should design a mechanism to reset the meta of this 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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] eolivelli merged pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


-- 
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] dlg99 commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java:
##########
@@ -295,6 +296,75 @@ public CompletableFuture<Void> clearPartitionedTopicTenantAsync(String tenant) {
             final String partitionedTopicPath = joinPath(PARTITIONED_TOPIC_PATH, tenant);
             return deleteIfExistsAsync(partitionedTopicPath);
         }
+
+        public CompletableFuture<Void> markPartitionedTopicDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                return CompletableFuture.completedFuture(null);
+            }
+            log.debug("markPartitionedTopicDeletedAsync {}", tn);

Review Comment:
   we don't really need isDebugEnabled() check here, the check is used to prevent boxing of primitives, tn is an Object already



-- 
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] michaeljmarshall commented on pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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

   @dlg99 - should this have gone to older release branches?


-- 
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] dlg99 commented on pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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

   > Two-phase delete, that's a good way to do it.
   
   Maybe after the https://github.com/apache/pulsar/pull/16590 done we can explore this option when it becomes available / stable. 
   
   Current approach should be good enough (and cherry-pickable into 2.10, if needed).
   
   
   
   


-- 
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] eolivelli commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java:
##########
@@ -295,6 +296,75 @@ public CompletableFuture<Void> clearPartitionedTopicTenantAsync(String tenant) {
             final String partitionedTopicPath = joinPath(PARTITIONED_TOPIC_PATH, tenant);
             return deleteIfExistsAsync(partitionedTopicPath);
         }
+
+        public CompletableFuture<Void> markPartitionedTopicDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                return CompletableFuture.completedFuture(null);
+            }
+            log.debug("markPartitionedTopicDeletedAsync {}", tn);

Review Comment:
   Please add guards as usual



-- 
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] eolivelli commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -2886,30 +2888,38 @@ public CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadata
                                     if (metadata.partitions == 0
                                             && !topicExists
                                             && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName, policies)
                                             && pulsar.getBrokerService()
                                                             .isDefaultTopicTypePartitioned(topicName, policies)) {
-
-                                        pulsar.getBrokerService()
-                                                .createDefaultPartitionedTopicAsync(topicName, policies)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    if (ex.getCause()
-                                                            instanceof MetadataStoreException.AlreadyExistsException) {
-                                                        // The partitioned topic might be created concurrently
-                                                        fetchPartitionedTopicMetadataAsync(topicName)
-                                                                .whenComplete((metadata2, ex2) -> {
-                                                                    if (ex2 == null) {
-                                                                        future.complete(metadata2);
-                                                                    } else {
-                                                                        future.completeExceptionally(ex2);
-                                                                    }
-                                                                });
-                                                    } else {
-                                                        future.completeExceptionally(ex);
-                                                    }
-                                                    return null;
-                                                });
+                                        isAllowAutoTopicCreationAsync(topicName, policies).thenAccept(allowed -> {

Review Comment:
   Cache stateless is a bigger problem  and I would keep that out of the scope this patch.
   This patch is following the way we do for all multi broker operations (like delete namespace)



-- 
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] codelipenghui commented on pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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

   @poorbarcode Please help review this PR. Thanks.


-- 
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] dlg99 commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java:
##########
@@ -295,6 +296,79 @@ public CompletableFuture<Void> clearPartitionedTopicTenantAsync(String tenant) {
             final String partitionedTopicPath = joinPath(PARTITIONED_TOPIC_PATH, tenant);
             return deleteIfExistsAsync(partitionedTopicPath);
         }
+
+        public CompletableFuture<Void> markPartitionedTopicDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                return CompletableFuture.completedFuture(null);
+            }
+            if (log.isDebugEnabled()) {
+                log.debug("markPartitionedTopicDeletedAsync {}", tn);
+            }
+            return updatePartitionedTopicAsync(tn, md -> {
+                md.deleted = true;
+                return md;
+            });
+        }
+
+        public CompletableFuture<Void> unmarkPartitionedTopicDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                return CompletableFuture.completedFuture(null);
+            }
+            if (log.isDebugEnabled()) {
+                log.debug("unmarkPartitionedTopicDeletedAsync {}", tn);
+            }
+            return updatePartitionedTopicAsync(tn, md -> {
+                md.deleted = false;
+                return md;
+            });
+        }
+
+        public CompletableFuture<Boolean> isPartitionedTopicBeingDeletedAsync(TopicName tn) {
+            if (tn.isPartitioned()) {
+                tn = TopicName.get(tn.getPartitionedTopicName());
+            }
+            return getPartitionedTopicMetadataAsync(tn)
+                    .thenApply(mdOpt -> mdOpt.map(partitionedTopicMetadata -> partitionedTopicMetadata.deleted)
+                            .orElse(false));
+        }
+
+        public CompletableFuture<Void> runWithMarkDeleteAsync(TopicName topic,
+                                                              Supplier<CompletableFuture<Void>> supplier) {
+            CompletableFuture<Void> future = new CompletableFuture<>();
+
+            markPartitionedTopicDeletedAsync(topic).whenCompleteAsync((markResult, markExc) -> {
+                final boolean mdFound;
+                if (markExc != null) {
+                    if (markExc.getCause() instanceof MetadataStoreException.NotFoundException) {
+                        mdFound = false;
+                    } else {
+                        log.error("Failed to mark the topic {} as deleted", topic, markExc);
+                        future.completeExceptionally(markExc);
+                        return;
+                    }
+                } else {
+                    mdFound = true;
+                }
+
+                supplier.get().whenComplete((deleteResult, deleteExc) -> {
+                    if (deleteExc != null && mdFound) {
+                        unmarkPartitionedTopicDeletedAsync(topic)

Review Comment:
   @poorbarcode can simply delete the topic again. 



-- 
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] dlg99 commented on a diff in pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -2886,30 +2888,38 @@ public CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadata
                                     if (metadata.partitions == 0
                                             && !topicExists
                                             && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName, policies)
                                             && pulsar.getBrokerService()
                                                             .isDefaultTopicTypePartitioned(topicName, policies)) {
-
-                                        pulsar.getBrokerService()
-                                                .createDefaultPartitionedTopicAsync(topicName, policies)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    if (ex.getCause()
-                                                            instanceof MetadataStoreException.AlreadyExistsException) {
-                                                        // The partitioned topic might be created concurrently
-                                                        fetchPartitionedTopicMetadataAsync(topicName)
-                                                                .whenComplete((metadata2, ex2) -> {
-                                                                    if (ex2 == null) {
-                                                                        future.complete(metadata2);
-                                                                    } else {
-                                                                        future.completeExceptionally(ex2);
-                                                                    }
-                                                                });
-                                                    } else {
-                                                        future.completeExceptionally(ex);
-                                                    }
-                                                    return null;
-                                                });
+                                        isAllowAutoTopicCreationAsync(topicName, policies).thenAccept(allowed -> {

Review Comment:
   @poorbarcode 
   createDefaultPartitionedTopicAsync calls createPartitionedTopicAsync which will produce AlreadyExistsException if PartitionedTopic metadata still exists (no matter the state of deleted property) and skip topic deletion/reread metadata. 
   
   It is ok if create tp_test-partion-0 already past the metadata check state. broker-1 issues delete tp_test-partion-0 after the PartitionedTopic metadata update (mark deleted) anyways. 



-- 
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] codecov-commenter commented on pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #18193:
URL: https://github.com/apache/pulsar/pull/18193#issuecomment-1291063390

   # [Codecov](https://codecov.io/gh/apache/pulsar/pull/18193?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#18193](https://codecov.io/gh/apache/pulsar/pull/18193?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (78b899e) into [master](https://codecov.io/gh/apache/pulsar/commit/6c65ca0d8a80bfaaa4d5869e0cea485f5c94369b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c65ca0) will **increase** coverage by `14.94%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pulsar/pull/18193/graphs/tree.svg?width=650&height=150&src=pr&token=acYqCpsK9J&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pulsar/pull/18193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #18193       +/-   ##
   =============================================
   + Coverage     34.91%   49.86%   +14.94%     
   - Complexity     5707     8279     +2572     
   =============================================
     Files           607      586       -21     
     Lines         53396    51366     -2030     
     Branches       5712     5494      -218     
   =============================================
   + Hits          18644    25613     +6969     
   + Misses        32119    22879     -9240     
   - Partials       2633     2874      +241     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `49.86% <0.00%> (+14.94%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pulsar/pull/18193?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...in/java/org/apache/pulsar/PulsarBrokerStarter.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhckJyb2tlclN0YXJ0ZXIuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [.../org/apache/pulsar/PulsarClusterMetadataSetup.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhckNsdXN0ZXJNZXRhZGF0YVNldHVwLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...g/apache/pulsar/PulsarClusterMetadataTeardown.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhckNsdXN0ZXJNZXRhZGF0YVRlYXJkb3duLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...org/apache/pulsar/PulsarInitialNamespaceSetup.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhckluaXRpYWxOYW1lc3BhY2VTZXR1cC5qYXZh) | `0.00% <ø> (ø)` | |
   | [.../main/java/org/apache/pulsar/PulsarStandalone.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhclN0YW5kYWxvbmUuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...ava/org/apache/pulsar/PulsarStandaloneBuilder.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhclN0YW5kYWxvbmVCdWlsZGVyLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...ava/org/apache/pulsar/PulsarStandaloneStarter.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL1B1bHNhclN0YW5kYWxvbmVTdGFydGVyLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...ersistentStickyKeyDispatcherMultipleConsumers.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL3BlcnNpc3RlbnQvUGVyc2lzdGVudFN0aWNreUtleURpc3BhdGNoZXJNdWx0aXBsZUNvbnN1bWVycy5qYXZh) | `44.71% <0.00%> (-11.06%)` | :arrow_down: |
   | [.../pulsar/broker/service/SharedConsumerAssignor.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL1NoYXJlZENvbnN1bWVyQXNzaWdub3IuamF2YQ==) | `61.11% <0.00%> (-7.41%)` | :arrow_down: |
   | [...er/systopic/NamespaceEventsSystemTopicFactory.java](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zeXN0b3BpYy9OYW1lc3BhY2VFdmVudHNTeXN0ZW1Ub3BpY0ZhY3RvcnkuamF2YQ==) | `73.33% <0.00%> (-5.24%)` | :arrow_down: |
   | ... and [222 more](https://codecov.io/gh/apache/pulsar/pull/18193/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


-- 
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] dlg99 commented on pull request #18193: [fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent.

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

   @poorbarcode @eolivelli Please take a look


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