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

[Java Client] Remove data race in MultiTopicsConsumerImpl to ensure correct message order #12456

Merged

Conversation

michaeljmarshall
Copy link
Member

@michaeljmarshall michaeljmarshall commented Oct 21, 2021

Motivation

@lhotari and I discovered a race condition in the MultiTopicsConsumerImpl<T> class. The race allows for messages to be delivered out of order.

We discovered the bug using the following steps:

  1. Create a 100 partition topic.
  2. Produce to the topic at 50k messages per second.
  3. Consume from the topic using a single, exclusive consumer in a single thread.
  4. Observe a small percentage of out of order messages (on average 13 in 100,000).

However, the race can happen with a single message and on a topic with a single partition.

The race comes in these two code blocks:

Message<T> message = incomingMessages.poll();
if (message == null) {
pendingReceives.add(result);
cancellationHandler.setCancelAction(() -> pendingReceives.remove(result));
} else {

// if asyncReceive is waiting : return message to callback without adding to incomingMessages queue
CompletableFuture<Message<T>> receivedFuture = nextPendingReceive();
if (receivedFuture != null) {
unAckedMessageTracker.add(topicMessage.getMessageId());
completePendingReceive(receivedFuture, topicMessage);
} else if (enqueueMessageAndCheckBatchReceive(topicMessage) && hasPendingBatchReceive()) {
notifyPendingBatchReceivedCallBack();
}

The first block is executed on the application's calling thread. The second block is executed in either the internalPinnedExecutor or the topic partition consumer's internalPinnedExecutor (these threads are not necessarily the same). As such, if the two blocks are called at the same time, it is possible for incomingMessages.poll(); to return a null result while nextPendingReceive(); also returns a null result. If this happens, the next state of the MultiTopicsConsumerImpl will be to have a single message in the incomingMessages queue and a single pending receive in the pendingReceives queue. Then, a message will deliver out of order.

This proposed solution follows the paradigm used by @Vanlightly in #11691. Essentially, the places where we need to inspect both the pendingReceives and the incomingMessages queues must be updated from a single thread: internalPinnedExecutor.

Modifications

  • Run the callback for consumer.receiveAsync() on the internalPinnedExecutor. I chose to run the whole callback on the internalPinnedExecutor instead of just the messageReceived method. If we left the callback using thenAccept and ran messageReceived on the internalPinnedExecutor, there is a chance that the callback will run on the calling thread, which is always the internalPinnedExecutor. That would mean that the messageReceived logic would actually run after the remaining callback logic that inspects the incomingMessages.size() and decides whether or not to pause the consumer. By scheduling the callback on the internalPinnedExecutor using thenAcceptAsync, we guarantee that the code is run together without the data race we're fixing in this PR.
  • Run MultiTopicsConsumerImpl#internalReceiveAsync on the internalPinnedExecutor.
  • Remove the checkState(message instanceof TopicMessageImpl); method call from the MultiTopicsConsumerImpl#internalReceiveAsync method. This decision may be controversial. I removed the check because we only ever add TopicMessageImpl to the MultiTopicsConsumerImpl's incomingMessages queue. If it is a necessary check, we could complete the future exceptionally when the message is not of type TopicMessageImpl.

Verifying this change

Since this is a fix for a data race, it is hard to test the change. I think the change is small enough that we don't need to add new tests for it, but please let me know if you think otherwise.

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

If yes was chosen, please highlight the changes

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API: (no)
  • The schema: (no)
  • The default values of configurations: (no)
  • The wire protocol: (no)
  • The rest endpoints: (no)
  • The admin cli options: (no)
  • Anything that affects deployment: (no)

Documentation

We should document this in release notes. No other docs need to be updated.

@eolivelli
Copy link
Contributor

@michaeljmarshall:Thanks for your contribution. For this PR, do we need to update docs?
(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

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.

Great catch!

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.

There is a test failure related to this change


org.apache.pulsar.client.impl.MultiTopicsConsumerImplTest.testReceiveAsyncCanBeCancelled(org.apache.pulsar.client.impl.MultiTopicsConsumerImplTest)
[INFO]   Run 1: PASS
Error:    Run 2: MultiTopicsConsumerImplTest.testReceiveAsyncCanBeCancelled:168 expected [true] but found [false]

@lhotari lhotari added the doc-not-needed Your PR changes do not impact docs label Oct 22, 2021
@eolivelli
Copy link
Contributor

@michaeljmarshall:Thanks for providing doc info!

@lhotari lhotari added release/2.8.2 release/2.9.0 type/bug The PR fixed a bug or issue reported a bug labels Oct 22, 2021
@lhotari lhotari added this to the 2.10.0 milestone Oct 22, 2021
@lhotari
Copy link
Member

lhotari commented Oct 22, 2021

There is a test failure related to this change


org.apache.pulsar.client.impl.MultiTopicsConsumerImplTest.testReceiveAsyncCanBeCancelled(org.apache.pulsar.client.impl.MultiTopicsConsumerImplTest)
[INFO]   Run 1: PASS
Error:    Run 2: MultiTopicsConsumerImplTest.testReceiveAsyncCanBeCancelled:168 expected [true] but found [false]

@michaeljmarshall the test should be fixed. This line


should be wrapped with Awaitility

        Awaitility.await().untilAsserted(() -> assertTrue(consumer.hasNextPendingReceive()));

@lhotari
Copy link
Member

lhotari commented Oct 22, 2021

Run the callback for consumer.receiveAsync() on the internalPinnedExecutor. I chose to run the whole callback on the internalPinnedExecutor instead of just the messageReceived method. If we left the callback using thenAccept and ran messageReceived on the internalPinnedExecutor, there is a chance that the callback will run on the calling thread, which is always the internalPinnedExecutor. That would mean that the messageReceived logic would actually run after the remaining callback logic that inspects the incomingMessages.size() and decides whether or not to pause the consumer. By scheduling the callback on the internalPinnedExecutor using thenAcceptAsync, we guarantee that the code is run together without the data race we're fixing in this PR.

Good catch @michaeljmarshall! Yes that's true that the other consumer will be another thread and thenAcceptAsync is needed to fix another potential race.

@michaeljmarshall
Copy link
Member Author

@lhotari - thank you for the solution. I ran the test locally without then with your proposed fix, and it works as expected.

lhotari
lhotari previously approved these changes Oct 22, 2021
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

@lhotari lhotari self-requested a review October 22, 2021 18:33
@lhotari lhotari dismissed their stale review October 22, 2021 19:07

I'm testing the changes and running into some problems, possibly performance issues.

@@ -269,7 +269,7 @@ private void receiveMessageFromConsumer(ConsumerImpl<T> consumer) {
// recursion and stack overflow
internalPinnedExecutor.execute(() -> receiveMessageFromConsumer(consumer));
Copy link
Member

Choose a reason for hiding this comment

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

Since the thenAcceptAsync now makes this code run on the internalPinnedExecutor, this extra scheduling to the internalPinnedExecutor should be removed. There isn't a risk of the stack growing infinitely since there is no direct recursive call chain.

This change improved the throughput performance by about 8% in a simple local benchmark.

# create topic with 10 partitions
./bin/pulsar-admin topics create-partitioned-topic -p 10 parttest
# run producer (a lot of small messages to stress test the logic)
./bin/pulsar-perf produce -ioThreads 4 -s 10 -o 6000 -p 200000 -r 350000 parttest
# run consumer (in another terminal)
./bin/pulsar-perf consume -q 6000 -p 200000 parttest

Throughput performance went from 290k msgs/s to about 315k msgs/s.

Copy link
Member Author

Choose a reason for hiding this comment

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

@lhotari - great catch. I just added a commit to remove the extra scheduling.

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.

Please address the comment about line 270.

@lhotari
Copy link
Member

lhotari commented Oct 23, 2021

@lhotari
Copy link
Member

lhotari commented Oct 25, 2021

@michaeljmarshall can you allow edits for maintainers for this PR? I'd like to push the test case (lhotari@1b0ad24b) to be part of this PR.

@michaeljmarshall
Copy link
Member Author

@michaeljmarshall can you allow edits for maintainers for this PR? I'd like to push the test case (lhotari@1b0ad24) to be part of this PR.

@lhotari - done, and I’ll address the rest of your comments soon.

@lhotari
Copy link
Member

lhotari commented Oct 25, 2021

@michaeljmarshall can you allow edits for maintainers for this PR? I'd like to push the test case (lhotari@1b0ad24) to be part of this PR.

@lhotari - done, and I’ll address the rest of your comments soon.

thanks @michaeljmarshall , I pushed a commit with the test to this PR now. Please pull the changes to your local PR branch.

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

@merlimat merlimat merged commit 6a2e3a1 into apache:master Oct 25, 2021
codelipenghui pushed a commit that referenced this pull request Oct 26, 2021
…orrect message order (#12456)

* [Java Client] Remove data race in MultiTopicsConsumerImpl to ensure correct message order

* Fix test

* Return the checkState method call to keep original behavior

* Reproduce out-of-order delivery issue in PR 12456

* Remove unnecessary scheduling of receiveMessageFromConsumer

Co-authored-by: Lari Hotari <lhotari@apache.org>
(cherry picked from commit 6a2e3a1)
@codelipenghui codelipenghui added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Oct 26, 2021
zeo1995 pushed a commit to zeo1995/pulsar that referenced this pull request Oct 26, 2021
* up/master:
  [C++] Fixed connection read error logging (apache#12492)
  [Pulsar SQL] Pulsar SQL support query big entry data (apache#12448)
  [Java Client] Remove data race in MultiTopicsConsumerImpl to ensure correct message order (apache#12456)
  Allow to have different instances LocalMemoryMetadataStore that share the same state (apache#12390)
  Remove unused ConsumerImpl.isTxnMessage (apache#12472)
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Oct 27, 2021
…orrect message order (apache#12456)

* [Java Client] Remove data race in MultiTopicsConsumerImpl to ensure correct message order

* Fix test

* Return the checkState method call to keep original behavior

* Reproduce out-of-order delivery issue in PR 12456

* Remove unnecessary scheduling of receiveMessageFromConsumer

Co-authored-by: Lari Hotari <lhotari@apache.org>
(cherry picked from commit 6a2e3a1)
(cherry picked from commit cfec2c9)
@michaeljmarshall michaeljmarshall deleted the fix-multi-topic-consumer-data-race branch October 28, 2021 05:45
@michaeljmarshall
Copy link
Member Author

@codelipenghui - would you please also cherry pick this to branch-2.9? Thanks.

@codelipenghui
Copy link
Contributor

@michaeljmarshall We can only cherry-pick it after 2.9.0 released.

lhotari added a commit to lhotari/nosqlbench that referenced this pull request Nov 11, 2021
- see apache/pulsar#12456 for details.
  - the bug applies to the use of receiveAsync and the workaround
    is to avoid the use of receiveAsync until the fix is included in the client
- receiveAsync was introduced previously to workaround the bug apache/pulsar#9921
  - that issue isn't critical
eolivelli pushed a commit to eolivelli/pulsar that referenced this pull request Nov 29, 2021
…orrect message order (apache#12456)

* [Java Client] Remove data race in MultiTopicsConsumerImpl to ensure correct message order

* Fix test

* Return the checkState method call to keep original behavior

* Reproduce out-of-order delivery issue in PR 12456

* Remove unnecessary scheduling of receiveMessageFromConsumer

Co-authored-by: Lari Hotari <lhotari@apache.org>
lhotari pushed a commit that referenced this pull request Dec 9, 2021
…orrect message order (#12456)

* [Java Client] Remove data race in MultiTopicsConsumerImpl to ensure correct message order

* Fix test

* Return the checkState method call to keep original behavior

* Reproduce out-of-order delivery issue in PR 12456

* Remove unnecessary scheduling of receiveMessageFromConsumer

Co-authored-by: Lari Hotari <lhotari@apache.org>
(cherry picked from commit 6a2e3a1)
@eolivelli eolivelli added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label Dec 15, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cherry-picked/branch-2.8 Archived: 2.8 is end of life cherry-picked/branch-2.9 Archived: 2.9 is end of life doc-not-needed Your PR changes do not impact docs release/2.8.2 release/2.9.1 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

7 participants