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

[Issue 11689][Client] Fixed block forever bug in Consumer.batchReceive #11691

Merged
merged 5 commits into from
Aug 23, 2021

Conversation

Vanlightly
Copy link
Contributor

@Vanlightly Vanlightly commented Aug 17, 2021

Fixes #11689

Motivation

When Consumer.batchReceive() is called concurrently by different threads there exists a race condition in ConsumerBase.java which when triggered causes a CompletableFuture in the queue pendingBatchReceives to be removed from the queue but not completed, causing the consumer to block forever. This has occurred a few times in production recently.

The issue is that there are concurrent calls to peek and poll in peekNextBatchReceive and the code is only correct when what is peeked is polled. If another thread calls poll between a peek and poll then this bug occurs. There is an error message when this occurs Bug: Removed entry wasn't the expected one.

Modifications

  • Put pinnedInternalExecutor in the ConsumerBase to allow batch timer, ConsumerImpl and MultiTopicsConsumerImpl to be able to submit work in a single threaded manner.
  • Ensure all poll() calls go through the pinnedInternalExecutor to avoid threading issues. The batch timer in ConsumerBase then simply submits its work to the executor from there.
  • Moved the failPendingReceive() to the ConsumerBase as it is the same for both.
  • Eagerly instantiate the pendingBatchReceives in ConsumerImpl and MultiTopicsConsumerImpl given that the lazy instantiation was not thread safe.
  • Replaced the use of peek for checking if a pending queue is empty or not.
  • Changed the peeking and polling method names to an iterator style of has and next.
  • The house keeping of clearing any completed futures from the queue is handled only by nextBatchReceive now.
  • Name changes caused some minor changes in some tests.
  • Some moving of imports due to checkstyle errors
  • Removed arguments from failPendingBatchReceives and failPendingReceives as they were superfluos.

Verifying this change

All existing tests that cover the consumer pass.

Reproduction was difficult, but I have seen it in production. The only way to make it trigger within a few seconds or minutes was to add a short thread sleep between the peek and poll while calling batchReceive concurrently from many threads with a non-zero maxNumMessages. I have run the repro steps in the issue with this fix and added a temporary thread sleep between peek and poll, and have verified it no longer occurs.

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

For contributor

Nothing to document here, purely internal implementation details.

For committer

For this PR, do we need to update docs?

  • If yes,

    • if you update docs in this PR, label this PR with the doc label.

    • if you plan to update docs later, label this PR with the doc-required label.

    • if you need help on updating docs, create a follow-up issue with the doc-required label.

  • If no, label this PR with the no-need-doc label and explain why.

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 added the type/bug The PR fixed a bug or issue reported a bug label Aug 17, 2021
@lhotari lhotari added this to the 2.9.0 milestone Aug 17, 2021
@lhotari lhotari requested a review from merlimat August 17, 2021 16:00
@lhotari
Copy link
Member

lhotari commented Aug 17, 2021

Good work @Vanlightly

@Anonymitaet Anonymitaet added the doc-not-needed Your PR changes do not impact docs label Aug 18, 2021
@315157973
Copy link
Contributor

The main reason for this to be locked is the following two pieces of logic:
pendingBatchReceiveTask is run by the timed thread pool of ConsumerBase, and pollNextBatchReceive needs to be called
notifyPendingBatchReceivedCallBack is run by the internalPinnedExecutor of each Consumer, and pollNextBatchReceive needs to be called

Can we set an abstract batchReceiveTimeout() on the base; So that peek and poll can be run in the internalPinnedExecutor, and there is no need to lock multiple times. Or any other suggestions?

@codelipenghui

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

@Vanlightly
Copy link
Contributor Author

@315157973 we could look at something like that. The ConsumerImpl has its internalPinnedExecutor so could be leveraged to avoid multi-threaded calls by submitting some part of the pendingBatchReceiveTask to it. But the MultiTopicsConsumerImpl doesn't do that, it uses the internal pool, rather than one pinned executor service. Perhaps the MultiTopicsConsumerImpl could also pin a single internal executor service as well? I don't know the history of the threading model choice so I'm not sure if that is a good way to go or not.

@codelipenghui
Copy link
Contributor

@315157973 @Vanlightly Thanks for your explanation, I like the approach @315157973 you have mentioned. We can create an abstract method in the ConsumerBase and for all the consumer implements ensure call the method by internalPinnedExecutor, and as @Vanlightly mentioned, the MultiTopicsConsumerImpl can follow the same way.

@Vanlightly
Copy link
Contributor Author

@315157973 @codelipenghui I have an implementation based on the suggestion, there was just one complication: failingPendingReceive().

On connectionOpened, closeAsync, unsubscribeAsync any pending receives are completed exceptionally. The ConsumerImpl submits the work to the internalPinnedExecutor, but the MultiTopicsConsumerImpl executes it synchronously.

The issues here:

  • all calls to failingPendingReceive() treat it as if it were synchronous code, even though it may be run asynchronously. I haven't been able to produce a problem, but theoretically, once closeAsync completes, there could still be pending receives that were not completed exceptionally yet, and shutdown() could throw them away, causing user code to block on batchReceive.
  • if we choose to run it synchronously then this makes the polling of pendingBatchReceives multi-threaded again - therefore needing the lock.
  • if we choose to run it asynchronously, then we need to make it return a CompletableFuture and ensure that connectionOpened, closeAsync, unsubscribeAsync treat it correctly as an asynchronous operation.

I have an implementation where failingPendingReceive() returns a CompletableFuture and all calling code treats it as an asynchronous call. The one thing I don't like is if user code calls shutdown(), then batchReceive can end up blocking forever again as the batchReceive future can get discarded.

I am in the process of running some nasty multi-threaded tests with concurrent calls to batchReceive and close, with one client, multiple shared consumers and multiple threads per consumer. Running for non-partitioned and partitioned topics. Will force push when I am happy there are no race conditions.

Ensure that all poll() calls to pendingBatchReceives
is done within the pinnedInternalExecutor to avoid a
race condition where a peek and a subsequent poll get
different pending receives.

Moved the pinnedInternalExecutor into the ConsumerBase
as both ConsumerImpl and MultiTopicsConsumerImpl require it.

failingPendingReceive() now always submits its work to the
internal executor returning a CompletableFuture and all callers
treat it as an asynchronous operation.
@Vanlightly
Copy link
Contributor Author

@315157973 @codelipenghui @lhotari I have pushed the change of ensuring all poll() calls go through the pinnedInternalExecutor to avoid threading issues. I didn't have to make an abstract method as I could simply move the pinnedInternalExecutor to ConsumerBase, given that both ConsumerImpl and MultiTopicsConsumerImpl require it. The batch timer in ConsumerBase then simply submits its work to the executor from there. I also moved the failPendingReceive() to the ConsumerBase as it is the same for both.

I also renamed the pinnedExecutor to externalPinnedExecutor as that is what it is and make it clearer.

Copy link
Contributor

@315157973 315157973 left a comment

Choose a reason for hiding this comment

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

LGTM

@@ -305,7 +305,7 @@ private void resumeReceivingFromPausedConsumersIfNeeded() {
break;
}

client.getInternalExecutorService().execute(() -> {
internalPinnedExecutor.execute(() -> {
Copy link
Contributor

@315157973 315157973 Aug 20, 2021

Choose a reason for hiding this comment

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

This seems to have changed the previous behavior. Before, you can select a different thread to execute each time, and now it has become one thread to execute.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes I mentioned this in my first comment on the suggested approach. If we're going to avoid all multi-threaded calls to to `pendingBatchReceives.poll()' then we can't use the internal pool here, we must use only one.

@315157973
Copy link
Contributor

@codelipenghui PTAL again

@codelipenghui
Copy link
Contributor

Thanks @Vanlightly The change looks good, I just left some minor comments about the internalPinnedExecutor and externalPinnedExecutor, PTAL.

Ensure that the externalPinnedExecutor is only called for user
code and internalPinnedExecutor used for internal tasks.
Some test refactoring to manage creation of executors.
@Vanlightly
Copy link
Contributor Author

/pulsarbot run-failure-checks

@Vanlightly
Copy link
Contributor Author

@codelipenghui I've made those updates, PTAL.

@jerrypeng jerrypeng merged commit bd942e1 into apache:master Aug 23, 2021
hangc0276 pushed a commit that referenced this pull request Aug 25, 2021
#11691)

* Fixed block forever bug in Consumer.batchReceive

Ensure that all poll() calls to pendingBatchReceives
is done within the pinnedInternalExecutor to avoid a
race condition where a peek and a subsequent poll get
different pending receives.

Moved the pinnedInternalExecutor into the ConsumerBase
as both ConsumerImpl and MultiTopicsConsumerImpl require it.

failingPendingReceive() now always submits its work to the
internal executor returning a CompletableFuture and all callers
treat it as an asynchronous operation.

* Fix broken MultiTopicsConsumerImplTest

Needed a real executor service to run the
failPendingReceive() method.

* Ensure all calls to messageReceived happen on internal executor

* Readd missing return statement in ConsumerImpl.closeAsync()

* Ensure correct usage of consumer internal executors

Ensure that the externalPinnedExecutor is only called for user
code and internalPinnedExecutor used for internal tasks.
Some test refactoring to manage creation of executors.

(cherry picked from commit bd942e1)
@hangc0276 hangc0276 added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Aug 25, 2021
dlg99 pushed a commit to dlg99/pulsar that referenced this pull request Sep 24, 2021
apache#11691)

* Fixed block forever bug in Consumer.batchReceive

Ensure that all poll() calls to pendingBatchReceives
is done within the pinnedInternalExecutor to avoid a
race condition where a peek and a subsequent poll get
different pending receives.

Moved the pinnedInternalExecutor into the ConsumerBase
as both ConsumerImpl and MultiTopicsConsumerImpl require it.

failingPendingReceive() now always submits its work to the
internal executor returning a CompletableFuture and all callers
treat it as an asynchronous operation.

* Fix broken MultiTopicsConsumerImplTest

Needed a real executor service to run the
failPendingReceive() method.

* Ensure all calls to messageReceived happen on internal executor

* Readd missing return statement in ConsumerImpl.closeAsync()

* Ensure correct usage of consumer internal executors

Ensure that the externalPinnedExecutor is only called for user
code and internalPinnedExecutor used for internal tasks.
Some test refactoring to manage creation of executors.
eolivelli pushed a commit to datastax/pulsar that referenced this pull request Sep 24, 2021
apache#11691)

* Fixed block forever bug in Consumer.batchReceive

Ensure that all poll() calls to pendingBatchReceives
is done within the pinnedInternalExecutor to avoid a
race condition where a peek and a subsequent poll get
different pending receives.

Moved the pinnedInternalExecutor into the ConsumerBase
as both ConsumerImpl and MultiTopicsConsumerImpl require it.

failingPendingReceive() now always submits its work to the
internal executor returning a CompletableFuture and all callers
treat it as an asynchronous operation.

* Fix broken MultiTopicsConsumerImplTest

Needed a real executor service to run the
failPendingReceive() method.

* Ensure all calls to messageReceived happen on internal executor

* Readd missing return statement in ConsumerImpl.closeAsync()

* Ensure correct usage of consumer internal executors

Ensure that the externalPinnedExecutor is only called for user
code and internalPinnedExecutor used for internal tasks.
Some test refactoring to manage creation of executors.
jerrypeng pushed a commit to jerrypeng/incubator-pulsar that referenced this pull request Nov 4, 2021
codelipenghui pushed a commit that referenced this pull request Dec 11, 2021
#11691)

* Fixed block forever bug in Consumer.batchReceive

Ensure that all poll() calls to pendingBatchReceives
is done within the pinnedInternalExecutor to avoid a
race condition where a peek and a subsequent poll get
different pending receives.

Moved the pinnedInternalExecutor into the ConsumerBase
as both ConsumerImpl and MultiTopicsConsumerImpl require it.

failingPendingReceive() now always submits its work to the
internal executor returning a CompletableFuture and all callers
treat it as an asynchronous operation.

* Fix broken MultiTopicsConsumerImplTest

Needed a real executor service to run the
failPendingReceive() method.

* Ensure all calls to messageReceived happen on internal executor

* Readd missing return statement in ConsumerImpl.closeAsync()

* Ensure correct usage of consumer internal executors

Ensure that the externalPinnedExecutor is only called for user
code and internalPinnedExecutor used for internal tasks.
Some test refactoring to manage creation of executors.

(cherry picked from commit bd942e1)
@codelipenghui codelipenghui added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Dec 11, 2021
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
apache#11691)

* Fixed block forever bug in Consumer.batchReceive

Ensure that all poll() calls to pendingBatchReceives
is done within the pinnedInternalExecutor to avoid a
race condition where a peek and a subsequent poll get
different pending receives.

Moved the pinnedInternalExecutor into the ConsumerBase
as both ConsumerImpl and MultiTopicsConsumerImpl require it.

failingPendingReceive() now always submits its work to the
internal executor returning a CompletableFuture and all callers
treat it as an asynchronous operation.

* Fix broken MultiTopicsConsumerImplTest

Needed a real executor service to run the
failPendingReceive() method.

* Ensure all calls to messageReceived happen on internal executor

* Readd missing return statement in ConsumerImpl.closeAsync()

* Ensure correct usage of consumer internal executors

Ensure that the externalPinnedExecutor is only called for user
code and internalPinnedExecutor used for internal tasks.
Some test refactoring to manage creation of executors.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cherry-picked/branch-2.7 Archived: 2.7 is end of life cherry-picked/branch-2.8 Archived: 2.8 is end of life doc-not-needed Your PR changes do not impact docs release/2.7.4 release/2.8.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.

Java consumer can block forever on batchReceive
8 participants