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 write get stuck when pipelined write is enabled #4143

Closed
wants to merge 2 commits into
base: master
from

Conversation

Projects
None yet
4 participants
@yiwu-arbug
Contributor

yiwu-arbug commented Jul 17, 2018

Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes #3704

Test Plan:
Run with the db_bench command mentioned in #3704 multiple times and make sure db_bench exit successfully. Also use the same command to verify the fix doesn't noticeably affect performance.

@yiwu-arbug yiwu-arbug requested a review from siying Jul 17, 2018

@yiwu-arbug yiwu-arbug requested a review from maysamyabandeh Jul 17, 2018

@facebook-github-bot

@yiwu-arbug has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

Writer dummy;
Writer* group_boundary = last_writer;
Writer* newest_writer = last_writer;
if (newest_writer_.compare_exchange_strong(newest_writer, &dummy)) {

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor

I guess "newest_writer_.compare_exchange_strong(last_writer, &dummy)" might be easier to read

// newest_writer. Otherwise, try to reset newest_writer_ to null.
bool has_pending_writer =
group_boundary != &dummy ||
!newest_writer_.compare_exchange_strong(newest_writer, nullptr);

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor
  1. I guess "newest_writer_.compare_exchange_strong(last_writer, nullptr)" might be easier to read
  2. I do not see the use of newest_writer variable here. If we get to this compare_exchange_strong, newest_writer is equal to last_writer.
  3. If above is correct, then we call the 2nd compare_exchange_strong only if the first one has failed before, and if the first one is failed already, meaning that newest_writer_ is no longer equal to last_writer, how could the 2nd compare_exchange_strong would work: if it was not equal the first time, it would not be equal in the 2nd time either!

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor

To make it easier to read, can you replace it with:

bool has_pending_writer = true; // When you define dummy
has_pending_writer = false; // When replacing with dummy works above;
if (!has_pending_writer) {
  has_pending_writer = newest_writer_.compare_exchange_strong(newest_writer, nullptr);
}

This comment has been minimized.

@yiwu-arbug

yiwu-arbug Jul 17, 2018

Contributor

Let me explain a bit more. There could be two cases on the first compare_exchange_strong:

  1. newest_writer_ == last_writer, in this case there isn't pending writers at that point.
    we append dummy after last_writer. newest_writer does not mean anything, but I set it to dummy for the second compare_exchange_strong.
  2. newest_writer_ != last_writer, in this case there's at least one pending writer.
    In this case newest_writer is set to be newest_writer_ (sorry for the confusing naming).

We need the second compare_exchange_strong only in case 1, where we are not sure if there's pending writer enter when we call LinkGroup. The second compare_exchange_strong is used to double check. In both cases, at the end newest_writer (no underscore) points to one of the pending writer, and we search for next leader from there.

This comment has been minimized.

@yiwu-arbug

yiwu-arbug Jul 17, 2018

Contributor

Regarding your comments:
2. newest_writer is used below at line 595. We search from newest_writer following link_older to find next leader.
3. actually we call the 2nd compare_exchange_strong only if the first one succeeded (case 1 above). In case 2 the first compare_exchange_strong already return a good pending writer and store in newest_writer.

@yiwu-arbug

This comment has been minimized.

Contributor

yiwu-arbug commented Jul 17, 2018

I updated the summary with an example to show how write can get stuck.

@yiwu-arbug yiwu-arbug force-pushed the yiwu-arbug:pipeline branch from 111ae7f to f080868 Jul 17, 2018

@facebook-github-bot

This comment has been minimized.

facebook-github-bot commented Jul 17, 2018

@yiwu-arbug has updated the pull request. Re-import the pull request

@facebook-github-bot

@yiwu-arbug has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

if (!has_dummy) {
// We find at least one pending writer when we insert dummy. We search
// for next leader from there.
next_leader = FindNextLeader(expected, last_writer);

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor

can we assert(next_leader != last_writer)

bool has_pending_writer =
!newest_writer_.compare_exchange_strong(expected, nullptr);
if (has_pending_writer) {
next_leader = FindNextLeader(expected, &dummy);

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor

can we assert(next_leader != &dummy)?

next_leader = next_leader->link_older;
assert(next_leader != nullptr);
// If we had insert dummy in the queue, remove it now and check if there

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor

do we actually remove the dummy here?
nit: s/had insert/have inserted/

This comment has been minimized.

@yiwu-arbug

yiwu-arbug Jul 17, 2018

Contributor

If the compare_exchange succeeded, dummy is removed; otherwise we remove it by resetting next_leader->link_older to null.

This comment has been minimized.

@maysamyabandeh

maysamyabandeh Jul 17, 2018

Contributor

dummy is removed

Which line does the removal?

@facebook-github-bot

This comment has been minimized.

facebook-github-bot commented Jul 17, 2018

@yiwu-arbug has updated the pull request. Re-import the pull request

@facebook-github-bot

@yiwu-arbug has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

riversand963 added a commit that referenced this pull request Jul 18, 2018

Fix write get stuck when pipelined write is enabled (#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes #3704
Pull Request resolved: #4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a
// so we know the boundary of the current write group.
Writer dummy;
Writer* expected = last_writer;
bool has_dummy = newest_writer_.compare_exchange_strong(expected, &dummy);

This comment has been minimized.

@siying

siying Jul 18, 2018

Contributor

By introducing this new check here, are we penalizing non-pipeline write's performance?

This comment has been minimized.

@yiwu-arbug

yiwu-arbug Jul 20, 2018

Contributor

No, this is in pipelined write only code path.

@yiwu-arbug yiwu-arbug deleted the yiwu-arbug:pipeline branch Jul 19, 2018

huachaohuang added a commit to huachaohuang/rocksdb that referenced this pull request Sep 3, 2018

Fix write get stuck when pipelined write is enabled (facebook#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes facebook#3704
Pull Request resolved: facebook#4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a

huachaohuang added a commit to huachaohuang/rocksdb that referenced this pull request Sep 3, 2018

Fix write get stuck when pipelined write is enabled (facebook#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes facebook#3704
Pull Request resolved: facebook#4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a

DorianZheng added a commit to DorianZheng/rocksdb that referenced this pull request Sep 5, 2018

Fix write get stuck when pipelined write is enabled (facebook#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes facebook#3704
Pull Request resolved: facebook#4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a

huachaohuang added a commit to huachaohuang/rocksdb that referenced this pull request Sep 11, 2018

Fix write get stuck when pipelined write is enabled (facebook#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes facebook#3704
Pull Request resolved: facebook#4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a

huachaohuang added a commit to huachaohuang/rocksdb that referenced this pull request Sep 11, 2018

Fix write get stuck when pipelined write is enabled (facebook#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes facebook#3704
Pull Request resolved: facebook#4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a

rcane added a commit to rcane/rocksdb that referenced this pull request Sep 13, 2018

Fix write get stuck when pipelined write is enabled (facebook#4143)
Summary:
Fix the issue when pipelined write is enabled, writers can get stuck indefinitely and not able to finish the write. It can show with the following example: Assume there are 4 writers W1, W2, W3, W4 (W1 is the first, W4 is the last).

T1: all writers pending in WAL writer queue:
WAL writer queue: W1, W2, W3, W4
memtable writer queue: empty

T2. W1 finish WAL writer and move to memtable writer queue:
WAL writer queue: W2, W3, W4,
memtable writer queue: W1

T3. W2 and W3 finish WAL write as a batch group. W2 enter ExitAsBatchGroupLeader and move the group to memtable writer queue, but before wake up next leader.
WAL writer queue: W4
memtable writer queue: W1, W2, W3

T4. W1, W2, W3 finish memtable write as a batch group. Note that W2 still in the previous ExitAsBatchGroupLeader, although W1 have done memtable write for W2.
WAL writer queue: W4
memtable writer queue: empty

T5. The thread corresponding to W3 create another writer W3' with the same address as W3.
WAL writer queue: W4, W3'
memtable writer queue: empty

T6. W2 continue with ExitAsBatchGroupLeader. Because the address of W3' is the same as W3, the last writer in its group, it thinks there are no pending writers, so it reset newest_writer_ to null, emptying the queue. W4 and W3' are deleted from the queue and will never be wake up.

The issue exists since pipelined write was introduced in 5.5.0.

Closes facebook#3704
Pull Request resolved: facebook#4143

Differential Revision: D8871599

Pulled By: yiwu-arbug

fbshipit-source-id: 3502674e51066a954a0660257e24ac588f815e2a
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment