Skip to content

Conversation

@dragosvictor
Copy link
Contributor

@dragosvictor dragosvictor commented Dec 6, 2023

PIP: PIP-307

Motivation

Topic unloading can be sped up, as described by PIP-307, by forwarding the target broker lookup data to clients (both producers and consumers). PR #21408 added support for the producers, this PR introduces similar functionality for the consumers.

Modifications

When the load balancer reassigns a bundle from an old 'source' broker to a new 'target' broker, the consumers are currently forcefully disconnected. The expectation is that they will issue topic lookup calls to locate the new broker.

This PR forwards this information to the consumers as part of the disconnect workflow, allowing them to connect directly to the target broker and skip the lookups in the process.

Specifically:

  • Added an optional assignedBrokerLookupData to the CloseConsumer command. The consumer uses this information to attempt to connect to the broker once. If the connection fails, it falls back to topic lookups.
  • When the ExtensibleLoadManager is unloading a bundle, the source broker first closes the topics without disconnecting the clients. When the bundle has been fully migrated to the target broker, the source broker closes the topics again, this time disconnecting the clients, along with sending the target broker information to the clients.
  • Message acknowledgments are silently ignored since the ledgers are being closed on the source broker too. When the topic loads up on the target broker, these in-progress messages will be re-delivered to the consumers.
  • Closing the dispatchers has the effect of disabling any further BookKeeper reads on the source broker. This is a desired outcome because these messages would not get acknowledged anyway. Inflight reads are still delivered but will have their acks ignored, as described above.

Verifying this change

  • Make sure that the change passes the CI checks.

This change added tests and can be verified as follows:

  • Extended existing unit tests testTransferClientReconnectionWithoutLookup and testUnloadClientReconnectionWithLookup to cover both producer and consumer behavior, as well as all subscription types. Ran the suite 100 times to make sure it was stable.
  • Manually verified the behavior using pulsar-perf in a k8s environment. Asserted that all messages pass through during a load balancing run, while no lookups are performed by the clients.

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment

Documentation

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

Matching PR in forked repository

PR in forked repository: (dragosvictor#1)

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Dec 6, 2023
@merlimat merlimat added this to the 3.2.0 milestone Dec 6, 2023
Copy link
Contributor

@heesung-sohn heesung-sohn left a comment

Choose a reason for hiding this comment

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

Also, I would like to confirm the following.

When topic.isTransferring() == true, we would like to implement the following consumer logic to make this topic transfer graceful.

  • ignore readEntry error (when the ledger is closed)
  • stop msg delivery to consumers (close subscription and dispatcher)
  • ignore in-flight and completed readEntry (do not send to the consumers)
  • ignore ack msg

Could you discuss if we further handle the above cases?

Copy link
Contributor

@heesung-sohn heesung-sohn left a comment

Choose a reason for hiding this comment

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

Hi, I think we covered 2 and 3 logic so far. Can we confirm if we covered 1 and 4 logics in this list?

  1. ignore readEntry error (when the ledger is closed)
  2. stop msg delivery to consumers (close subscription and dispatcher)
  3. ignore in-flight and completed readEntry (do not send to the consumers)
  4. ignore ack msg

@dragosvictor dragosvictor reopened this Dec 8, 2023
Copy link
Contributor

@heesung-sohn heesung-sohn left a comment

Choose a reason for hiding this comment

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

LGTM. Thanks.

Copy link
Contributor

@gaoran10 gaoran10 left a comment

Choose a reason for hiding this comment

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

Great work! Left some comments.

Copy link
Member

@Demogorgon314 Demogorgon314 left a comment

Choose a reason for hiding this comment

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

Great work!

if (consumerFuture != null && consumerFuture.isDone() && !consumerFuture.isCompletedExceptionally()) {
Consumer consumer = consumerFuture.getNow(null);
Subscription subscription = consumer.getSubscription();
if (subscription.getTopic().isTransferring()) {
Copy link
Member

Choose a reason for hiding this comment

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

Do we have a test to cover in transferring acks?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not specifically, but the modified test covers ack behavior: https://github.com/apache/pulsar/pull/21682/files#diff-744119c61c9f6a1b786c3966acd1e0f63748985cf68a6a1a15418c8d9900a9a8R543-R547. The test does not succeed unless all messages sent during the transfer surface out on the consumers. Since there are 200 messages involved, at least one of them likely has the acknowledgement initially ignored.

@dragosvictor dragosvictor changed the title [improve][broker] PIP-307: Skip consumer topic lookups during topic load balancing [improve][broker] PIP-307: Implement broker and client consumer changes when topic is unloading Dec 13, 2023
…ersistent/PersistentTopic.java

Co-authored-by: Kai Wang <kwang@streamnative.io>
…ersistent/PersistentDispatcherMultipleConsumers.java

Co-authored-by: Penghui Li <penghui@apache.org>
@lhotari
Copy link
Member

lhotari commented Dec 19, 2023

It looks like this PR broke the test SubscriptionMessageDispatchThrottlingTest.testClosingRateLimiter (#21756). The fix is in #21736 . @dragosvictor @heesung-sn @gaoran10 Please review.

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 ready-to-test

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants