Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix] Combination of autocreate + forced delete of partitioned topic with active consumer leaves topic metadata inconsistent. #17308

Merged
merged 3 commits into from
Sep 1, 2022

Conversation

dlg99
Copy link
Contributor

@dlg99 dlg99 commented Aug 26, 2022

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)

Modifications

Don't allow deletion in this case until all consumers/producers disconnected.

I experimented with option to not allow autocreate after deletion of partitioned topics but that ends up in the reasons that led to #14920 + tricky corner cases between metadata updates.

Verifying this change

This change added tests

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)

  • doc-not-needed

Bug fix

  • doc
    (Your PR contains doc changes)

  • doc-complete
    (Docs have been already added)

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Aug 26, 2022
…ith active consumer leaves topic metadata inconsistent.
@dlg99 dlg99 force-pushed the master_delete_fails branch from 98e6e79 to ba6a0b1 Compare August 26, 2022 22:51
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch.

I left a comment about the error code. I think we should return something that is more user friendly as an Internal Error seems that the server is broken.

I have another thought, but I am not sure how we can fix it:
How can we help a user that doesn't have control on client applications but needs to delete a partitioned topic?
Maybe this is not a big deal because if you want to have control over the topics you turn off automatic creation at namespace level

try {
admin.topics().deletePartitionedTopic(topic, true);
fail("expected error because partitioned topic has active producer");
} catch (PulsarAdminException.ServerSideErrorException e) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ServerSideError is not very user friendly.
I would expect 'Conflict' or the same thing that happens when you do not use the 'force' option

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@eolivelli I return the same TopicBusyException as other cases when topic deletion is not possible.
If it needs to be translated into some other error for pulsar admin/rest API I suggest we create a bug and resolve this in a different PR

if (isClosingOrDeleting) {
log.warn("[{}] Topic is already being closed or deleted", topic);
return FutureUtil.failedFuture(new TopicFencedException("Topic is already fenced"));
} else if (failIfHasSubscriptions && !subscriptions.isEmpty()) {
return FutureUtil.failedFuture(
new TopicBusyException("Topic has subscriptions: " + subscriptions.keys()));
} else if (failIfHasBacklogs && hasBacklogs()) {
List<String> backlogSubs =
subscriptions.values().stream()
.filter(sub -> sub.getNumberOfEntriesInBacklog(false) > 0)
.map(PersistentSubscription::getName).toList();
return FutureUtil.failedFuture(
new TopicBusyException("Topic has subscriptions did not catch up: " + backlogSubs));
} else if (TopicName.get(topic).isPartitioned()
&& (getProducers().size() > 0 || getNumberOfConsumers() > 0)
&& getBrokerService().isAllowAutoTopicCreation(topic)) {
// to avoid inconsistent metadata as a result
return FutureUtil.failedFuture(
new TopicBusyException("Partitioned topic has active consumers or producers and "
+ "auto-creation of topic is allowed"));

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lgtm

@eolivelli
Copy link
Contributor

(As discussed offline) As a follow up work we should disable automatic topic creation before deleting a namespace, this way we will allow users to delete namespaces and tenants more easily

@@ -1139,6 +1139,13 @@ private CompletableFuture<Void> delete(boolean failIfHasSubscriptions,
.map(PersistentSubscription::getName).toList();
return FutureUtil.failedFuture(
new TopicBusyException("Topic has subscriptions did not catch up: " + backlogSubs));
} else if (TopicName.get(topic).isPartitioned()
&& (getProducers().size() > 0 || getNumberOfConsumers() > 0)
&& getBrokerService().isAllowAutoTopicCreation(topic)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This PR can not solve every scenario:

The cmd delete topic is executed between these two instructions: consumer lookup and consumer subscribe, even if the partitioned topic is deleted successfully, but the client already has the topic-meta(which has been deleted), then the consumer subscribes with the topic name "topic-partition-x". You can reproduce like this:

1. create partitioned topic "tp_test"

2. consumer lookup

3. delete topic

4. consumer subscribe

5."tp_test-partition-x" created

@eolivelli
Copy link
Contributor

@poorbarcode this is a good catch indeed!

Thinking more about this case the "problem" is related to the sequences of things that happen in the two operations:

  • delete a partitioned topic
  • automatic creation of a partitioned topic

We must prevent these two things to be performed concurrently

Ideally the two operations should be performed in reverse order:
when you delete the topic the last thing that you delete is the first thing that "create partitioned topic" checks
so until the deletion is finished a new creation cannot fails

@eolivelli
Copy link
Contributor

btw I think that this patch mitigates most of the usecases that happen in real world (and we saw it a lot of them in some production clusters after upgrading to 2.10)

@dlg99
Copy link
Contributor Author

dlg99 commented Aug 31, 2022

I also think this change mitigates the problems we've encountered in prod; before implementing this I experimented with different approach (as mentioned in the description of the PR)

@eolivelli eolivelli merged commit 9529850 into apache:master Sep 1, 2022
eolivelli pushed a commit to datastax/pulsar that referenced this pull request Sep 5, 2022
…with active consumer leaves topic metadata inconsistent. (apache#17308)

(cherry picked from commit 9529850)
eolivelli pushed a commit to datastax/pulsar that referenced this pull request Sep 5, 2022
…with active consumer leaves topic metadata inconsistent. (apache#17308)

(cherry picked from commit 9529850)
nodece pushed a commit to nodece/pulsar that referenced this pull request Sep 8, 2022
…with active consumer leaves topic metadata inconsistent. (apache#17308)
@poorbarcode
Copy link
Contributor

poorbarcode commented Sep 9, 2022

Hi @eolivelli @dlg99

I think we should revert this PR.

System topics are always created automatically, and isAllowAutoTopicCreation(system topic) always returns true. Users will not stop the system topic manually before deleting a namespace. If a namespace contains any partitioned system topic, this namespace can not be deleted( After this PR is Merged ).

We should consider other ways to solve this problem

@eolivelli
Copy link
Contributor

@poorbarcode you are right, I think that we must revert this patch.
Let me send a PR

eolivelli added a commit to eolivelli/pulsar that referenced this pull request Sep 9, 2022
…d topic with active consumer leaves topic metadata inconsistent. (apache#17308)"

This reverts commit 9529850.
nicoloboschi added a commit to datastax/pulsar that referenced this pull request Sep 9, 2022
…d topic with active consumer leaves topic metadata inconsistent. (apache#17308)"

This reverts commit 9524418.
eolivelli added a commit that referenced this pull request Sep 9, 2022
…d topic with active consumer leaves topic metadata inconsistent. (#17308)" (#17566)

This reverts commit 9529850.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
doc-not-needed Your PR changes do not impact docs
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants