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 another deadlock which can occur while acquiring merge buffers #16372

Merged
merged 7 commits into from
May 8, 2024

Conversation

LakshSingla
Copy link
Contributor

@LakshSingla LakshSingla commented May 2, 2024

Description

This PR fixes up a deadlock that was introduced by #15420.

Consider the following sequence of calls:

Time Thread1 Thread2
1 pool.reserve(query1)
2 pool.reserve(query2)
3 pool.clean(query1)

Assuming that the pool has enough merge buffers to allow each query to proceed individually, but not together, the following will happen (in the original code). The clean(at t=3, by thread1) would be blocked on the reserve(at t=2, by thread2) to complete. However, the reserve won't succeed since it doesn't have enough merge buffers, and is waiting on the clean call, hence the cyclic dependency. (NOTE: This also assumes that the resourceId of query1 and query2 would hold the lock on the same entry in the concurrent hash map).

The fix is to reserve the buffer before acquiring a lock on the concurrent hash map and hold the locks for the minimal time possible. This way, other calls like remove() and fetch() are not blocked on the actual acquisition of the merge buffers to succeed.

I have added a test case to replicate the sequence of events. The test case fails before the patch, however succeeds after the patch. The test case isn't perfect, however, given the blocking nature of the reserve() call, I couldn't find an alternative.

Thanks @weishiuntsai and @gianm for identifying and debugging the test case.

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.

@LakshSingla LakshSingla added this to the 30.0.0 milestone May 2, 2024
* This test assumes a few things about the implementation of the interfaces, which are laid out in the comments.
* <p>
* The test should complete under 10 seconds, and the majority of the time would be consumed by waiting for the thread
* that sleeps for 5 seconds
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think we should add new unit tests that have sleeps; the test suite takes long enough to run already. They are also a sign of a test that is not robust.

Is it possible to rewrite this test to not use a sleep? If not, I'd suggest having it be @Ignore so it doesn't run on every test suite run. Then include a comment in the code of GroupByResourcesReservationPool itself that says if a future developer is changing the logic, they should run this test manually to ensure they aren't introducing a deadlock.

Copy link
Contributor Author

@LakshSingla LakshSingla May 2, 2024

Choose a reason for hiding this comment

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

Is it possible to rewrite this test to not use a sleep?

The problem I am running into at this point is that I want to signal to Thread1 that Thread2 has called the reserve() operation, however, the reserve() itself is blocking. I have tried the polling approach, using synchronized blocks and the current method, however they all run into the same blocker - there's no way to signal from a thread that it has called a blocking operation (before its completion). Any suggestions on how I can achieve this?

Else I'll annotate the test with @Ignore

@@ -104,18 +104,23 @@ public GroupByResourcesReservationPool(
}

/**
* Reserves appropriate resources, and maps it to the queryResourceId (usually the query's resource id) in the internal map
* Reserves appropriate resources, and maps it to the queryResourceId (usually the query's resource id) in the internal map.
* This is a blocking call, and can block upto the given query's timeout
Copy link
Contributor

Choose a reason for hiding this comment

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

up to (spelling)

pool.compute(queryResourceId, (id, existingResource) -> {
if (existingResource != null) {
resources.close();
Copy link
Contributor

Choose a reason for hiding this comment

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

Rather than risk resources.close() holding the compute lock for too long, how about using pool.putIfAbsent instead, and then if putIfAbsent returns nonnull (signifying there was some existing resource), then call resources.close() and throw the defensive exception.

Copy link
Contributor Author

@LakshSingla LakshSingla May 3, 2024

Choose a reason for hiding this comment

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

The change that you suggested seems much cleaner.

While testing if the method throws, I found that the method can prematurely block instead of throwing with the duplicate query id exception. This is because we are allocating the resources first, and then entering it into the map. While we never expect duplicate query resource IDs to be present, we still want the defensive check to be thrown as soon as possible, instead of being blocked for the merge buffers to be free (note: it's not a deadlock, but an inconvenience).

I have made some changes to alleviate this issue.

@LakshSingla
Copy link
Contributor Author

@gianm Thanks for the review. I have updated the PR with the suggested changes (and a few more). I wasn't able to make the test work without having a Sleep, so I have annotated it with @Ignore.

@LakshSingla LakshSingla closed this May 3, 2024
@LakshSingla LakshSingla reopened this May 3, 2024
if (resourcesReference != null) {
GroupByQueryResources resource = resourcesReference.get();
// Reference should refer to a non-empty resource
assert resource != null;
Copy link
Contributor

@gianm gianm May 6, 2024

Choose a reason for hiding this comment

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

Better to use DruidException.defensive than assert. The main benefit of assert is the checks are omitted when running for real (unless -ea is provided), which can be useful for performance reasons. But that's not a consideration here, really. The check is very cheap compared to the surrounding code, and it'd be better to always do it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Since all the handling was done in a single class, I figured it would be fine to have an assert statement. Modified it to DruidException.defensive().

}

// We have reserved a spot in the map. Now begin the blocking call.
GroupByQueryResources resources =
Copy link
Contributor

Choose a reason for hiding this comment

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

What happens if prepareResource fails, for example due to timeout? Will the reference be cleaned up from the map somehow?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Great catch, it would have polluted the map. I have modified the code accordingly.

// We have reserved a spot in the map. Now begin the blocking call.
resources = GroupingEngine.prepareResource(groupByQuery, mergeBufferPool, willMergeRunner, groupByQueryConfig);
}
catch (Exception e) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you change this to Throwable? It isn't likely that we will actually get a Throwable here that is not an Exception, but IMO it's good practice for "definitely must happen" cleanup-on-failure code to catch Throwable rather than Exception. It makes it before more like a finally or try-with-resources, both of which would activate on any Throwable.

Copy link
Contributor

@gianm gianm left a comment

Choose a reason for hiding this comment

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

Approved but please consider change the catch to catch Throwable.

@LakshSingla
Copy link
Contributor Author

@gianm I have made the change. Thanks for the review!

@LakshSingla LakshSingla merged commit dded473 into apache:master May 8, 2024
87 checks passed
@LakshSingla LakshSingla deleted the deadlock-2 branch May 8, 2024 12:34
adarshsanjeev pushed a commit to adarshsanjeev/druid that referenced this pull request May 10, 2024
adarshsanjeev added a commit that referenced this pull request May 10, 2024
…16372) (#16427)

Fixes a deadlock while acquiring merge buffers

Co-authored-by: Laksh Singla <lakshsingla@gmail.com>
gianm pushed a commit to gianm/druid that referenced this pull request May 10, 2024
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.

3 participants