Skip to content

disable broker parallel merge for Java versions greater than 8 and less than 20 due to apparent bug#14130

Closed
clintropolis wants to merge 8 commits intoapache:masterfrom
clintropolis:broker-parallel-merge-broken-in-some-java-versions
Closed

disable broker parallel merge for Java versions greater than 8 and less than 20 due to apparent bug#14130
clintropolis wants to merge 8 commits intoapache:masterfrom
clintropolis:broker-parallel-merge-broken-in-some-java-versions

Conversation

@clintropolis
Copy link
Member

@clintropolis clintropolis commented Apr 20, 2023

Description

Through a bit of luck, stumbled into a possible bug with broker parallel merging on certain java versions which can eventually result in all worker threads becoming completely blocked effectively blocking queries.

I have adjusted the ParallelMergeCombiningSequenceTest to check that the pool becomes idle after a short period whenever an exception occurs. The adjust tests fail pretty consistently for me with java 11 and 17, but can repeat indefinitely on java 8 and java 20 on my laptop. Some of these timeout tests are still ever so slightly racy since the exception message can vary slightly depending on which part in the code triggers the timeout exception, but the true important part is that previously missing a check that the pool becomes idle.

I still haven't determined why this happens, perhaps a java bug or at least some issue with how we are trying to configure/use the pool on these versions. It could also just be an artifact of how the test is written, and not a real issue? Im not really sure yet.

Release note

Broker parallel merge is now disabled for Java versions greater than 8 and less than 20 due to a bug with how this mechanism operates on these versions that can result in all pool threads becoming blocked effectively halting the ability to process most queries. Java 8 and Java 20 do not appear to exhibit these symptoms and so still have parallel result merging enabled by default.


This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • a release note entry in the PR description.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

@clintropolis
Copy link
Member Author

I guess this should probably just disable by default instead of completely, to allow risk takers to still enable it for these versions. I'll modify this PR in a bit to change it to just be off by default, and issue a warning in the logs if it detects the config as enabled for these java versions to perhaps more carefully monitor things to ensure the brokers are ok.


public boolean useParallelMergePool()
{
if (JvmUtils.majorVersion() < 20 && JvmUtils.majorVersion() >= 9) {
Copy link
Contributor

Choose a reason for hiding this comment

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

A comment here explaining the reasoning would be nice.

if (hasTimeout) {
final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime();
if (thisTimeoutNanos < 0) {
item = null;
Copy link
Contributor

Choose a reason for hiding this comment

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

What's this line & similar ones like it for?

Copy link
Member Author

Choose a reason for hiding this comment

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

these changes were just me be cautious to help ensure that isReleasable always can return true if an exception is thrown. Im not sure if these are actually problematic or fix any problems, but based on reading the fork join pool code and looking at how it does the managed blocking stuff there were a handful of places where it is checked on a loop which made me a bit nervous, so I wanted to try to minimize the chances of anything getting stuck.

count++;
if (count % batchSize == 0) {
yield();
this.yield();
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this change cosmetic or does it change behavior?

Copy link
Member Author

Choose a reason for hiding this comment

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

i think cosmetic, intellij was highlighting and complaining about it for something that didn't really seem applicable:

Screenshot 2023-04-21 at 12 21 38 PM

and suggested changing it to this.yield so i just did it 🤷

public boolean isReleasable()
{
return resultBatch != null && !resultBatch.isDrained();
return yielder.isDone() || (resultBatch != null && !resultBatch.isDrained());
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this fixing something?

Copy link
Member Author

Choose a reason for hiding this comment

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

same comment about being overly cautious

assertException(input);
Throwable t = Assert.assertThrows(RuntimeException.class, () -> assertException(input));
Assert.assertEquals("exploded", t.getMessage());
Assert.assertTrue(pool.awaitQuiescence(1, TimeUnit.SECONDS));
Copy link
Contributor

Choose a reason for hiding this comment

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

suggestion— put the wait-for-quiescence stuff in @After

@clintropolis
Copy link
Member Author

fyi, I am still trying to prove this is actually real and not an artifact of how the test is written

@vogievetsky vogievetsky removed this from the 26.0 milestone May 6, 2023
@jasonk000
Copy link
Contributor

jasonk000 commented May 15, 2023

@clintropolis I took a look at this and I think there is a an assumption that is broken by the testing configuration but that could occur in the real world. It's late here so I'll share my notes and wrap up.

From the user's perspective, the query is canceled. However, for the system, (I think) the yielder keeps reading data at least until it can load a batch of data.

In short I think there is something where the two-layer yielders do not propagate cancellation when the exception occurs, so the one started on the fjpool keeps running.

If I extend the timeout and capture a stack trace,


"ForkJoinPool-1-worker-5" #28 daemon prio=5 os_prio=0 cpu=16.62ms elapsed=8.44s tid=0x00007fe044003800 nid=0x7700 waiting on condition  [0x00007fe08ebe0000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
	at java.lang.Thread.sleep(java.base@11.0.18/Native Method)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequenceTest$2$1.next(ParallelMergeCombiningSequenceTest.java:903)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequenceTest$2$1.next(ParallelMergeCombiningSequenceTest.java:876)
	at org.apache.druid.java.util.common.guava.BaseSequence.toYielder(BaseSequence.java:82)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$ResultBatch.fromSequence(ParallelMergeCombiningSequence.java:892)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$SequenceBatcher.block(ParallelMergeCombiningSequence.java:943)
	at java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.18/ForkJoinPool.java:3118)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$SequenceBatcher.getBatchYielder(ParallelMergeCombiningSequence.java:931)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$YielderBatchedResultsCursor.initialize(ParallelMergeCombiningSequence.java:1039)
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$PrepareMergeCombineInputsAction.compute(ParallelMergeCombiningSequence.java:742)
	at java.util.concurrent.RecursiveAction.exec(java.base@11.0.18/RecursiveAction.java:189)
	at java.util.concurrent.ForkJoinTask.doExec(java.base@11.0.18/ForkJoinTask.java:290)
	at java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(java.base@11.0.18/ForkJoinPool.java:1020)
	at java.util.concurrent.ForkJoinPool.scan(java.base@11.0.18/ForkJoinPool.java:1656)
	at java.util.concurrent.ForkJoinPool.runWorker(java.base@11.0.18/ForkJoinPool.java:1594)
	at java.util.concurrent.ForkJoinWorkerThread.run(java.base@11.0.18/ForkJoinWorkerThread.java:183)

And adding a whole ton of println / statements I get the stdout, in which we can see:

  • yielder 0 at ParallelMergeCombiningSequence.toYielder
  • yielder 1 at ParallelMergeCombiningSequence$ResultBatch.fromSequence
  • during initialize(), make() is called, which starts loading values
  • yielder 0 sees the QueryTimeoutException
  • yielder 1 keeps generating values

I haven't quite gotten to the bottom of it, but there's something here, I think it's vaguely in this area:

In the test:

  • we create a number of sequences, including a blocking sequence:
    input.add(blockingSequence(someSize, 400, 500, 1, 500, true));
  • we expect that the blocking sequence will run for longer than the given query timeout, so the query timeout will throw an exception
  • the flow creates a ParallelMergeCombiningSequence, and a timer is set up
  • when toYielder is called on the flow it creates a sub-action and yielder, the parent yielder has a timeout / cancellation
  • since the first round of loading values (up to 2048?) is too long in duration, given 1-500ms blocking, we never get enough values to complete a full first batch to complete a yield
  • so the cancellation on yielder 1 never gets to a point of being checked, it's stuck in initialize, at no point during the initialize call do we have an opportunity to bail out

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