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 3070: Fix bug where checkAllLedgers gets stuck when read throttling is enabled #3214

Merged
merged 4 commits into from Sep 29, 2022

Conversation

massakam
Copy link

Motivation

The read throttling feature in Auditor#checkAllLedgers added by #2973, but when it was enabled checkAllLedgers got stuck.

The cause is that, as mentioned in #3070, all the BookKeeperClientWorker-OrderedExecutor threads wait for the semaphore to be released, and there is no thread that can execute EntryExistsCallback#readEntryComplete where the semaphore is released.

Changes

Add a new thread pool to the LedgerChecker class. The BookKeeperClientWorker-OrderedExecutor threads threads delegate subsequent processing (this includes waiting for the semaphore to be released) to threads in this thread pool as soon as they release the semaphore.

This will prevent the BookKeeperClientWorker-OrderedExecutor from getting stuck due to waiting for the semaphore to be released.

Master Issue: #3070

@massakam
Copy link
Author

rerun failure checks

@dlg99
Copy link
Contributor

dlg99 commented Apr 26, 2022

@massakam is it possible to avoid the extra executor?

I think the problem is that checkFragments/verifyLedgerFragment tries to acquire a permit from a callback in bookieClient.readEntry call; but there is already a permit request before that call. I think we can simply add a flag to not request/release extra permit in checkFragments/verifyLedgerFragment call chain in this case.

@massakam massakam force-pushed the fix-checkAllLedgers branch 3 times, most recently from 2a1a927 to 5071cf5 Compare April 26, 2022 08:34
@massakam
Copy link
Author

rerun failure checks

3 similar comments
@massakam
Copy link
Author

rerun failure checks

@massakam
Copy link
Author

rerun failure checks

@massakam
Copy link
Author

rerun failure checks

@massakam
Copy link
Author

@dlg99 Changed the implementation to achieve throttling using RateLimiter instead of Semaphore. What do you think?

I think this is simpler than the implementation of adding a new flag and requesting/releasing a permit according to its value.

@gaozhangmin
Copy link
Contributor

@dlg99 Changed the implementation to achieve throttling using RateLimiter instead of Semaphore. What do you think?

I think this is simpler than the implementation of adding a new flag and requesting/releasing a permit according to its value.

+1

@massakam
Copy link
Author

@dlg99 PTAL

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

@dlg99
Copy link
Contributor

dlg99 commented Jun 21, 2022

@massakam I think that Throttle is not a good approach here.

Requests-in-progress (RIPs) approach is very similar when everything works but easier to tune and self-adjusts to remote system's dynamically changing capabilities while throttle is impossible to tuen for that.
Imagine you configured 20 RIPs and one request takes 100ms normally. Now you have 200 RPS (requests per second).
Remote system (bookie) slowed down and now take 500ms to process a request, now you are down to 40RPS without overloading the bookie, changing config etc.
Bookie speeds up and can process request in 10ms - the rate dynamically goes up to 2000rps.

Let's use throttle now: set it to 200 request per second.
In the normal case everything looks ok and the same as in normal case above.
Now the bookie slows down.
Throttle keeps on letting through 200 RPS even though the bookie can only handle 40, with that requests will pile up and eventually make result in OOM on bookie or client side.
Bookie speeds up and can process request in 10ms - the rate stays at 200rps.

I'd prefer to track down where the semaphore is not released and fix that.

Throttle is simple, does not require release etc. but it has its drawback and was not used there for a reason.

@massakam
Copy link
Author

@dlg99 Hmm... you have a point there.

However, I once tried to fix it in the way you suggested first, but I gave it up because the code became complicated and there was a high risk of missing the release of the acquired permit. In this approach, if an additional read is required in EntryExistsCallback#readEntryComplete(), it is necessary not to release the permit immediately, but after the subsequent read is complete.

I think it would be better to move to throttling with a rate limiter rather than to keep the RIPs approach and leaving the possibility of getting stuck.

@massakam
Copy link
Author

Another option I have is to revert to the first approach with the extra executor.

@@ -213,7 +213,7 @@ public class ServerConfiguration extends AbstractConfiguration<ServerConfigurati
"auditorMaxNumberOfConcurrentOpenLedgerOperations";
protected static final String AUDITOR_ACQUIRE_CONCURRENT_OPEN_LEDGER_OPERATIONS_TIMEOUT_MSEC =
"auditorAcquireConcurrentOpenLedgerOperationsTimeOutMSec";
protected static final String IN_FLIGHT_READ_ENTRY_NUM_IN_LEDGER_CHECKER = "inFlightReadEntryNumInLedgerChecker";
protected static final String READ_ENTRY_RATE_IN_LEDGER_CHECKER = "readEntryRateInLedgerChecker";
Copy link
Contributor

Choose a reason for hiding this comment

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

The PR #2973 has been released in BookKeeper 4.15.0, we'd better mark the old one as deprecated instead of removing it. And set the old value to the new one if the user sets the old one.

Copy link
Author

Choose a reason for hiding this comment

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

@hangc0276 I don't think the value of inFlightReadEntryNumInLedgerChecker should be set to readEntryRateInLedgerChecker, as the two settings have slightly different meanings. Therefore, I decided to restore the old setting. If both readEntryRateInLedgerChecker and inFlightReadEntryNumInLedgerChecker are set, readEntryRateInLedgerChecker takes precedence and inFlightReadEntryNumInLedgerChecker is ignored.

@massakam massakam force-pushed the fix-checkAllLedgers branch 2 times, most recently from a36ff10 to 36fd148 Compare June 25, 2022 06:30
@massakam
Copy link
Author

PTAL

@massakam
Copy link
Author

massakam commented Jul 6, 2022

ping @dlg99 @hangc0276

@massakam
Copy link
Author

@dlg99

This gist https://gist.github.com/dlg99/505849e1010a20c6d439ecd53f500a85 is more of a demo of what's going on than actual fix (after all, it breaks testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble - might be a test issue).

I have a question about this gist. Why is releasePermit() added on line 79 of LedgerChecker.java?
https://gist.github.com/dlg99/505849e1010a20c6d439ecd53f500a85#file-pr3214-diff-L79

EntryExistsCallback#readEntryComplete also calls releasePermit(), so it seems that one extra permit is released.

And if releasePermit() on line 75 of LedgerChecker.java is removed, TestLedgerChecker times out.

@massakam
Copy link
Author

rerun failure checks

2 similar comments
@massakam
Copy link
Author

rerun failure checks

@massakam
Copy link
Author

rerun failure checks

@dlg99
Copy link
Contributor

dlg99 commented Sep 21, 2022

@massakam

Why is releasePermit() added on line 79 of LedgerChecker.java?

It is in the callback.
Permits requested before bookieClient.readEntry called, and must be released in the callback passed there.

                final long entryToRead = curEntryId;

                final EntryExistsCallback eecb = new EntryExistsCallback(lh.getLedgerMetadata().getWriteQuorumSize(),
                                              new GenericCallback<Boolean>() {
                                                  @Override
                                                  public void operationComplete(int rc, Boolean result) {
                                                      releasePermit();
                                                      if (result) {
                                                          fragments.add(lastLedgerFragment);
                                                      }
                                                      checkFragments(fragments, cb,
                                                          percentageOfLedgerFragmentToBeVerified);
                                                  }
                                              });

                try {
                    DistributionSchedule.WriteSet writeSet = lh.getDistributionSchedule().getWriteSet(entryToRead);
                    acquirePermits(writeSet.size());
                    for (int i = 0; i < writeSet.size(); i++) {
                            BookieId addr = curEnsemble.get(writeSet.get(i));
                            bookieClient.readEntry(addr, lh.getId(), entryToRead,
                                    eecb, null, BookieProtocol.FLAG_NONE);
                    }
                    writeSet.recycle();
                } catch (InterruptedException e) {
                    LOG.error("InterruptedException when checking entry : {}", entryToRead, e);
                }

@massakam
Copy link
Author

massakam commented Sep 21, 2022

@dlg99 That callback method is called from EntryExistsCallback#readEntryComplete. releasePermit() is executed numReads times in EntryExistsCallback#readEntryComplete, so releasePermit() will be executed numReads + 1 times in total.

    private class EntryExistsCallback implements ReadEntryCallback {
        AtomicBoolean entryMayExist = new AtomicBoolean(false);
        final AtomicInteger numReads;
        final GenericCallback<Boolean> cb;

        EntryExistsCallback(int numReads,
                            GenericCallback<Boolean> cb) {
            this.numReads = new AtomicInteger(numReads);
            this.cb = cb;
        }

        @Override
        public void readEntryComplete(int rc, long ledgerId, long entryId,
                                      ByteBuf buffer, Object ctx) {
            releasePermit(); // Executed `numReads` times
            if (BKException.Code.NoSuchEntryException != rc && BKException.Code.NoSuchLedgerExistsException != rc
                    && BKException.Code.NoSuchLedgerExistsOnMetadataServerException != rc) {
                entryMayExist.set(true);
            }

            if (numReads.decrementAndGet() == 0) {
                cb.operationComplete(rc, entryMayExist.get()); // Call `GenericCallback#operationComplete` and execute `releasePermit()` one more time
            }
        }
    }

In fact, when I logged the value of semaphore.availablePermits() after the test was completed, it was inFlightReadEntryNum + 1. But it should return to inFlightReadEntryNum.

@dlg99
Copy link
Contributor

dlg99 commented Sep 22, 2022

@massakam you are absolutely right, that was an extra release.
So, we should not call checkFragments() in the callback that runs on the bookie's ordered executor (acquire blocks a chance to run of the bookie callback that releases permit).
https://gist.github.com/dlg99/3e1524f0804c4688ccb31e3300b89c4e basically your initial idea with the executor, needs other tests to close LedgerChecker.
This one: https://gist.github.com/dlg99/28f07f3bcd3f71000b1faba6b906fb09 is kind of the same but uses common pool executor (via whenCompleteAsync)

@massakam
Copy link
Author

rerun failure checks

2 similar comments
@massakam
Copy link
Author

rerun failure checks

@massakam
Copy link
Author

rerun failure checks

Copy link
Contributor

@dlg99 dlg99 left a comment

Choose a reason for hiding this comment

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

Thank you!

@massakam
Copy link
Author

This one: https://gist.github.com/dlg99/28f07f3bcd3f71000b1faba6b906fb09 is kind of the same but uses common pool executor (via whenCompleteAsync)

Adopted this approach and modified the code.

@massakam
Copy link
Author

@hangc0276 @rdhabalia @eolivelli @StevenLuMT PTAL. And if there is no problem, please merge this.

@eolivelli eolivelli merged commit 21560fb into apache:master Sep 29, 2022
@eolivelli
Copy link
Contributor

@massakam
Merged, thanks !

@massakam massakam deleted the fix-checkAllLedgers branch September 30, 2022 01:50
@hangc0276 hangc0276 modified the milestones: 4.17.0, 4.16.0 Oct 14, 2022
zymap pushed a commit that referenced this pull request Oct 26, 2022
hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Nov 7, 2022
hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Nov 7, 2022
nicoloboschi pushed a commit to datastax/bookkeeper that referenced this pull request Jan 11, 2023
…ling is enabled (apache#3214)

(cherry picked from commit 21560fb)
(cherry picked from commit 376040c)
hangc0276 pushed a commit that referenced this pull request Feb 9, 2023
…is enabled (#3655)

### Motivation

Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. #3214). That PR was merged so we cherry-picked it and released it, but `checkAllLedgers` still got stuck when the `inFlightReadEntryNumInLedgerChecker` value was very small.

The cause is that multiple `BookKeeperClientWorker-OrderedExecutor` threads are blocked waiting for the semaphore to be released. This semaphore is released by the `BookKeeperClientWorker-OrderedExecutor` threads, so they should not run the `LedgerChecker#checkLedger` method.
```
"BookKeeperClientWorker-OrderedExecutor-1-0" #60 prio=5 os_prio=0 tid=0x00007f680401d800 nid=0x75e8 waiting on condition [0x00007f684a6e9000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000a2121780> (a java.util.concurrent.Semaphore$NonfairSync)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
        at java.util.concurrent.Semaphore.acquire(Semaphore.java:467)
        at org.apache.bookkeeper.client.LedgerChecker.acquirePermit(LedgerChecker.java:163)
        at org.apache.bookkeeper.client.LedgerChecker.checkLedger(LedgerChecker.java:431)
        at org.apache.bookkeeper.replication.Auditor.lambda$null$4(Auditor.java:1216)
        at org.apache.bookkeeper.replication.Auditor$$Lambda$137/1553334572.openComplete(Unknown Source)
        at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232)
        at org.apache.bookkeeper.client.LedgerOpenOp$2.readLastConfirmedComplete(LedgerOpenOp.java:218)
        at org.apache.bookkeeper.client.LedgerHandle$14.getLacComplete(LedgerHandle.java:1718)
        at org.apache.bookkeeper.client.PendingReadLacOp.readLacComplete(PendingReadLacOp.java:154)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion$1.readLacComplete(PerChannelBookieClient.java:1801)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion.handleV3Response(PerChannelBookieClient.java:1840)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1473)
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:750)
```

### Changes

In the `Auditor` class, have the `LedgerChecker#checkLedger` method run in a different thread from the `BookKeeperClientWorker-OrderedExecutor` threads.

Master Issue: #3070
zymap pushed a commit that referenced this pull request Feb 16, 2023
…is enabled (#3655)

### Motivation

Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. #3214). That PR was merged so we cherry-picked it and released it, but `checkAllLedgers` still got stuck when the `inFlightReadEntryNumInLedgerChecker` value was very small.

The cause is that multiple `BookKeeperClientWorker-OrderedExecutor` threads are blocked waiting for the semaphore to be released. This semaphore is released by the `BookKeeperClientWorker-OrderedExecutor` threads, so they should not run the `LedgerChecker#checkLedger` method.
```
"BookKeeperClientWorker-OrderedExecutor-1-0" #60 prio=5 os_prio=0 tid=0x00007f680401d800 nid=0x75e8 waiting on condition [0x00007f684a6e9000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000a2121780> (a java.util.concurrent.Semaphore$NonfairSync)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
        at java.util.concurrent.Semaphore.acquire(Semaphore.java:467)
        at org.apache.bookkeeper.client.LedgerChecker.acquirePermit(LedgerChecker.java:163)
        at org.apache.bookkeeper.client.LedgerChecker.checkLedger(LedgerChecker.java:431)
        at org.apache.bookkeeper.replication.Auditor.lambda$null$4(Auditor.java:1216)
        at org.apache.bookkeeper.replication.Auditor$$Lambda$137/1553334572.openComplete(Unknown Source)
        at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232)
        at org.apache.bookkeeper.client.LedgerOpenOp$2.readLastConfirmedComplete(LedgerOpenOp.java:218)
        at org.apache.bookkeeper.client.LedgerHandle$14.getLacComplete(LedgerHandle.java:1718)
        at org.apache.bookkeeper.client.PendingReadLacOp.readLacComplete(PendingReadLacOp.java:154)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion$1.readLacComplete(PerChannelBookieClient.java:1801)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion.handleV3Response(PerChannelBookieClient.java:1840)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1473)
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:750)
```

### Changes

In the `Auditor` class, have the `LedgerChecker#checkLedger` method run in a different thread from the `BookKeeperClientWorker-OrderedExecutor` threads.

Master Issue: #3070

(cherry picked from commit 4ca4b4e)
hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Jun 26, 2023
…is enabled (apache#3655)

Some time ago, I fixed a bug where check of all ledgers periodically run by the auditor got stuck (cf. apache#3214). That PR was merged so we cherry-picked it and released it, but `checkAllLedgers` still got stuck when the `inFlightReadEntryNumInLedgerChecker` value was very small.

The cause is that multiple `BookKeeperClientWorker-OrderedExecutor` threads are blocked waiting for the semaphore to be released. This semaphore is released by the `BookKeeperClientWorker-OrderedExecutor` threads, so they should not run the `LedgerChecker#checkLedger` method.
```
"BookKeeperClientWorker-OrderedExecutor-1-0" apache#60 prio=5 os_prio=0 tid=0x00007f680401d800 nid=0x75e8 waiting on condition [0x00007f684a6e9000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000a2121780> (a java.util.concurrent.Semaphore$NonfairSync)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
        at java.util.concurrent.Semaphore.acquire(Semaphore.java:467)
        at org.apache.bookkeeper.client.LedgerChecker.acquirePermit(LedgerChecker.java:163)
        at org.apache.bookkeeper.client.LedgerChecker.checkLedger(LedgerChecker.java:431)
        at org.apache.bookkeeper.replication.Auditor.lambda$null$4(Auditor.java:1216)
        at org.apache.bookkeeper.replication.Auditor$$Lambda$137/1553334572.openComplete(Unknown Source)
        at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232)
        at org.apache.bookkeeper.client.LedgerOpenOp$2.readLastConfirmedComplete(LedgerOpenOp.java:218)
        at org.apache.bookkeeper.client.LedgerHandle$14.getLacComplete(LedgerHandle.java:1718)
        at org.apache.bookkeeper.client.PendingReadLacOp.readLacComplete(PendingReadLacOp.java:154)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion$1.readLacComplete(PerChannelBookieClient.java:1801)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadLacCompletion.handleV3Response(PerChannelBookieClient.java:1840)
        at org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1473)
        at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:750)
```

In the `Auditor` class, have the `LedgerChecker#checkLedger` method run in a different thread from the `BookKeeperClientWorker-OrderedExecutor` threads.

Master Issue: apache#3070

(cherry picked from commit 4ca4b4e)
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.

None yet

8 participants