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

Avoid deadlocks on JOIN Engine tables #29544

Merged
merged 8 commits into from
Oct 12, 2021

Conversation

Algunenano
Copy link
Member

Changelog category (leave one):

  • Bug Fix (user-visible misbehaviour in official stable or prestable release)

Changelog entry (a user-readable short description of the changes that goes to CHANGELOG.md):
Avoid deadlocks when reading and writting on JOIN Engine tables at the same time

Detailed description / Documentation draft:

The patch ended up requiring way more changes that I wanted to. I decided to replace the shared_mutex by a RWLock for several reasons:

  • The main one because it allows getting a READ lock after you've gotten one already even if there has been a request by other request/thread to get a write lock in the meantime. This was what caused the deadlock at Deadlock with JOIN Engine #29485
  • RWLock has profile events, so it's easier to track
  • RWLock has timeouts, which is a nice way to avoid deadlocks requiring a database restart (as you can't kill a query that's forever waiting for a mutex).

Since RWLock requires a query_id to work effectively I had to add a context parameter to multiple places. The 2 functions that I didn't add it and instead used RWLockImpl::NO_QUERY were totalRows and totalBytes as their declaration is used in 10 other places, but it could be done if we wanted to.

The added test fails all the time in my system pre-change (usually even in the second iteration of the loop) but I had to increase the size the table until if was reliable deadlocking the database as it's purely timebased.

Fixes #29485
The bug affects all stable releases so it would be great to have it backported to 21.8+ at least. Let me know if I can help there if the proposed solution is accepted.

@robot-clickhouse robot-clickhouse added the pr-bugfix Pull request with bugfix, not backported by default label Sep 29, 2021
@vdimir vdimir self-assigned this Sep 30, 2021
Copy link
Member

@vdimir vdimir left a comment

Choose a reason for hiding this comment

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

Test 01732_race_condition_storage_join_long.sh is interesting with this changes. Now all queries in this test should success.
Would some of them fail with timeout error after change?

Comment on lines 748 to 749
throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates",
ErrorCodes::LOGICAL_ERROR);
Copy link
Member

Choose a reason for hiding this comment

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

It's not changed in this PR, but I wonder why it's LOGICAL_ERROR, can it be reached in we perform INSERT into JOIN Engine table during SELECT?

UPD: because lock already acquired in StorageJoin::insertBlock, but storage_join_lock can be set only from StorageJoin::getJoinLocked that are used for SELECTing data.

@Algunenano
Copy link
Member Author

Test 01732_race_condition_storage_join_long.sh is interesting with this changes. Now all queries in this test should success.
Would some of them fail with timeout error after change?

Before the change (master) the queries would get stuck forever if a deadlock happens (read starts and gets lock, writes starts and asks for write lock, first read tries to get another read lock). The lock_acquire_timeout setting doesn't have any effect in master because it uses a shared mutex that doesn't respect the timeout.

With this change, the issue should not appear anymore as RWLock allows you to jump the lock queue is you already hold a read lock and request another. This is what fixes the bug. The timeouts only happen in the tests because I set it too low for the sanitizers but if they were large enough it should eventually go through. I'm not testing that the timeout occurs for low enough values of lock_acquire_timeout because there isn't a reliable way that I know to trigger it.

@Algunenano
Copy link
Member Author

Ups I see that you were talking about a different test and not the one added in the PR.

Would some of them fail with timeout error after change?

Not unless the server is extremely slow for other reasons and I would say that would be a desirable outcome (respect the settings instead of blocking forever). In the test itself there are only 4 concurrent queries (3 reads and an insert) so I think it's highly unlikely that they would need to wait for 120 to acquire the lock.

@vdimir
Copy link
Member

vdimir commented Oct 1, 2021

Not unless the server is extremely slow for other reasons and I would say that would be a desirable outcome (respect the settings instead of blocking forever). In the test itself there are only 4 concurrent queries (3 reads and an insert) so I think it's highly unlikely that they would need to wait for 120 to acquire the lock.

If I'm not mistaken lock is acquired for the whole SELECT query, we get StorageJoin in HashJoin that holds lock during execution. For INSERT lock is acquired per each block.

Another question does RWLock used only because it supports timeouts? Can problem be solved with std::timed_mutex, for instance? As #29485 (comment) says RWLock is a bit specific.

Finally, have you research how difficult to solve problem without timeout? Like to lock mutex in correct order, change logic a bit? What is the main issue here?

@Algunenano
Copy link
Member Author

If I'm not mistaken lock is acquired for the whole SELECT query, we get StorageJoin in HashJoin that holds lock during execution. For INSERT lock is acquired per each block.

For SELECT queries the lock is acquired multiple times, the first one I think happens during InterpreterSelectQuery, and then when read() is called. Might be more.

Another question does RWLock used only because it supports timeouts? Can problem be solved with std::timed_mutex, for instance? As #29485 (comment) says RWLock is a bit specific.

No, the main reason was to avoid the deadlock due to acquiring the same lock multiple times (and somebody else gets in between). shared_timed_mutex would not fix that. Anything else that allows something like this would work in this situation.

Finally, have you research how difficult to solve problem without timeout? Like to lock mutex in correct order, change logic a bit? What is the main issue here?

I think it's doable, ideally by just holding the mutex at the start (InterpreterSelectQuery) and never acquire it again under any circumstance. 2 super tricky situations for show how hard that would be:

  • If you use joinGet or JOIN multiple times in different subqueries you would need to make sure you only try to hold the mutex once, and not call it again as otherwise there can be a race condition with a write query between them that can't be fixed by std mutexes (that I know of).
  • If you use joinGet over a table and, at the same time, read the size of that table via system.tables you would also try to hold the mutex twice, one at the start and one at runtime as far as I can see. Not sure what would happen if you do several queries concurrently doing an INSERT to the join table from that.

So, from what's available I couldn't see a better option; but I don't say there isn't.

@Algunenano
Copy link
Member Author

I can't access the logs of the failed tests as they point to an internal URL. Are they related to changes in this PR?

@vdimir
Copy link
Member

vdimir commented Oct 7, 2021

I can't access the logs of the failed tests as they point to an internal URL. Are they related to changes in this PR?

Seems that it isn't task just timed out (I see similar issue in other PRs).

No, the main reason was to avoid the deadlock due to acquiring the same lock multiple times (and somebody else gets in between). shared_timed_mutex would not fix that. Anything else that allows something like this would work in this situation.

What's about std::recursive_timed_mutex ?

@Algunenano
Copy link
Member Author

Seems that it isn't task just timed out (I see similar issue in other PRs).

Thanks!

What's about std::recursive_timed_mutex ?

It only provides exclusive ownership only so you wouldn't be able to have multiple read queries running at the same time.

@vdimir
Copy link
Member

vdimir commented Oct 11, 2021

Functional stateless tests flaky check (address)
02033_join_engine_deadlock
Test runs too long (> 60s). Make it faster.

Let's mark it as long

@Algunenano
Copy link
Member Author

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
pr-bugfix Pull request with bugfix, not backported by default
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Deadlock with JOIN Engine
3 participants