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] [broker] Close dispatchers stuck due to mismatch between dispatcher.consumerList and dispatcher.consumerSet #22270

Merged
merged 5 commits into from
Mar 23, 2024

Conversation

poorbarcode
Copy link
Contributor

Motivation

2024-03-13T07:47:36,071+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://{tenat}/{ns}/{topic name}] Global topic inactive for 86400 seconds, closed repl producers
2024-03-13T07:47:36,073+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=5, consumerName=082c9, address=/127.0.0.6:34293}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=10, consumerName=f0c65, address=/127.0.0.6:32911}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - Removed consumer Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=5, consumerName=7f82f, address=/127.0.0.6:39763} with pending 0 acks
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - [persistent://{tenat}/{ns}/{topic name} / {subscription}] Trying to remove a non-connected consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=10, consumerName=f0c65, address=/127.0.0.6:32911}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=5, consumerName=7f82f, address=/127.0.0.6:39763}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - Removed consumer Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=18, consumerName=1aebe, address=/127.0.0.6:33045} with pending 0 acks
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=18, consumerName=b44be, address=/127.0.0.6:44907}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - [persistent://{tenat}/{ns}/{topic name} / {subscription}] Trying to remove a non-connected consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=5, consumerName=082c9, address=/127.0.0.6:34293}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=10, consumerName=a2fc6, address=/127.0.0.6:54177}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - [persistent://{tenat}/{ns}/{topic name} / {subscription}] Trying to remove a non-connected consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=18, consumerName=b44be, address=/127.0.0.6:44907}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=18, consumerName=1aebe, address=/127.0.0.6:33045}
2024-03-13T07:47:36,074+0000 [pulsar-inactivity-monitor-25-1] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - Removed consumer Consumer{subscription=PersistentSubscription{topic=persistent://{tenat}/{ns}/{topic name}, name={subscription}}, consumerId=10, consumerName=a2fc6, address=/127.0.0.6:54177} with pending 0 acks
2024-03-13T07:47:36,186+0000 [pulsar-io-4-22] WARN  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://{tenat}/{ns}/{topic name}] Attempting to subscribe to a fenced topic

The logs were printed on our cluster(2.11.2)

  • The consumer {clientCnx 127.0.0.6:44907, consumerId: 18} has been removed twice when calling dispatcher.disconnectAllConsumers.[1]
  • Pulsar printed Trying to remove a non-connected consumer when removing the consumer {clientCnx 127.0.0.6:44907, consumerId: 18} at the second time, which means this consumer is duplicated in dispatcher.consumerList.[2]
  • Once the log "Trying to remove a non-connected consumer" is printed, the closeFuture of dispatcher will never be completed, leading to the process of topic.delete being stuck.
  • The topic stays in a fenced state.

[1]: dispatcher.disconnectAllConsumers

public synchronized CompletableFuture<Void> disconnectAllConsumers() {
  consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData));
}

[2]: dispatcher.removeConsumer

public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException {
    // decrement unack-message count for removed consumer
    addUnAckedMessages(-consumer.getUnackedMessages());
    if (consumerSet.removeAll(consumer) == 1) {
        consumerList.remove(consumer);
        log.info("Removed consumer {} with pending {} acks", consumer, consumer.getPendingAcks().size());
        if (consumerList.isEmpty()) {
            if (closeFuture != null) {
                closeFuture.complete(null); // Once the log "Trying to remove a non-connected consumer" was printed, it means the closeFuture will never be completed.
            }
        } 
    } else {
        log.info("[{}] Trying to remove a non-connected consumer: {}", name, consumer);
    }
}

Modifications

I haven't found the logic of repeating consumer in dispatcher.consumerList yet(it may caused by a consumer that has been added twice, and dispatcher.consumerSet merged two adding), I just fixed the follow-up issue.

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository: x

@poorbarcode poorbarcode added type/bug The PR fixed a bug or issue reported a bug release/3.0.4 release/3.2.2 release/2.11.5 labels Mar 14, 2024
@poorbarcode poorbarcode added this to the 3.3.0 milestone Mar 14, 2024
@poorbarcode poorbarcode self-assigned this Mar 14, 2024
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Mar 14, 2024
@poorbarcode poorbarcode changed the title [fix] [broker] Delete topic stuck due to mismatch between dispatcher.consumerList and dispatcher.consumerSet [fix] [broker] Close dispatchers stuck due to mismatch between dispatcher.consumerList and dispatcher.consumerSet Mar 15, 2024
Copy link
Contributor

@hanmz hanmz left a comment

Choose a reason for hiding this comment

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

LGTM

@poorbarcode poorbarcode requested a review from lhotari March 18, 2024 15:47
Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

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

LGTM

@codecov-commenter
Copy link

codecov-commenter commented Mar 19, 2024

Codecov Report

Attention: Patch coverage is 92.30769% with 1 lines in your changes are missing coverage. Please review.

Project coverage is 73.68%. Comparing base (bbc6224) to head (4882977).
Report is 75 commits behind head on master.

Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #22270      +/-   ##
============================================
+ Coverage     73.57%   73.68%   +0.10%     
- Complexity    32624    32802     +178     
============================================
  Files          1877     1887      +10     
  Lines        139502   139488      -14     
  Branches      15299    15293       -6     
============================================
+ Hits         102638   102778     +140     
+ Misses        28908    28750     -158     
- Partials       7956     7960       +4     
Flag Coverage Δ
inttests 26.58% <46.15%> (+2.00%) ⬆️
systests 24.54% <46.15%> (+0.22%) ⬆️
unittests 72.95% <92.30%> (+0.11%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

Files Coverage Δ
...sistent/PersistentDispatcherMultipleConsumers.java 74.52% <92.30%> (+0.20%) ⬆️

... and 103 files with indirect coverage changes

Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

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

It looks like we don't need this PR if we follow this comment to fix the issue. I have tested on my laptop.

@poorbarcode
Copy link
Contributor Author

It looks like we don't need this PR if we follow this #22283 (review) to fix the issue. I have tested on my laptop.

Changed the implementation: Just add a defensive code to avoid the topic not being unloaded anymore: remove the consumers which are not mismatched between consumerSet and consumerList, and print an error log

@poorbarcode
Copy link
Contributor Author

/pulsarbot rerun-failure-checks

@poorbarcode poorbarcode merged commit cba1600 into apache:master Mar 23, 2024
55 checks passed
poorbarcode added a commit that referenced this pull request Mar 26, 2024
…cher.consumerList and dispatcher.consumerSet (#22270)

(cherry picked from commit cba1600)
poorbarcode added a commit that referenced this pull request Mar 26, 2024
…cher.consumerList and dispatcher.consumerSet (#22270)

(cherry picked from commit cba1600)
poorbarcode added a commit that referenced this pull request Mar 26, 2024
…cher.consumerList and dispatcher.consumerSet (#22270)

(cherry picked from commit cba1600)
poorbarcode added a commit that referenced this pull request Mar 26, 2024
…cher.consumerList and dispatcher.consumerSet (#22270)

(cherry picked from commit cba1600)
mukesh-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 15, 2024
…cher.consumerList and dispatcher.consumerSet (apache#22270)

(cherry picked from commit cba1600)
(cherry picked from commit 94edfe4)
mukesh-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 17, 2024
…cher.consumerList and dispatcher.consumerSet (apache#22270)

(cherry picked from commit cba1600)
(cherry picked from commit 94edfe4)
mukesh-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 17, 2024
…cher.consumerList and dispatcher.consumerSet (apache#22270)

(cherry picked from commit cba1600)
(cherry picked from commit 94edfe4)
mukesh-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 19, 2024
…cher.consumerList and dispatcher.consumerSet (apache#22270)

(cherry picked from commit cba1600)
(cherry picked from commit 94edfe4)
srinath-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 23, 2024
…cher.consumerList and dispatcher.consumerSet (apache#22270)

(cherry picked from commit cba1600)
(cherry picked from commit 94edfe4)
nodece pushed a commit to ascentstream/pulsar that referenced this pull request May 13, 2024
…cher.consumerList and dispatcher.consumerSet (apache#22270)

(cherry picked from commit cba1600)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants