-
Notifications
You must be signed in to change notification settings - Fork 963
Fix core dumps triggered by rocksdb compacting when shutdown bk #4706
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,12 +20,14 @@ | |
| */ | ||
| package org.apache.bookkeeper.bookie.storage.ldb; | ||
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.google.common.collect.Iterables; | ||
| import java.io.Closeable; | ||
| import java.io.IOException; | ||
| import java.util.Map.Entry; | ||
| import java.util.Set; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import org.apache.bookkeeper.bookie.Bookie; | ||
| import org.apache.bookkeeper.bookie.EntryLocation; | ||
| import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.Batch; | ||
|
|
@@ -48,7 +50,8 @@ public class EntryLocationIndex implements Closeable { | |
| private final KeyValueStorage locationsDb; | ||
| private final ConcurrentLongHashSet deletedLedgers = ConcurrentLongHashSet.newBuilder().build(); | ||
| private final EntryLocationIndexStats stats; | ||
| private boolean isCompacting; | ||
| @VisibleForTesting | ||
| final AtomicBoolean compacting = new AtomicBoolean(false); | ||
|
|
||
| public EntryLocationIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath, | ||
| StatsLogger stats) throws IOException { | ||
|
|
@@ -67,7 +70,19 @@ public EntryLocationIndex(ServerConfiguration conf, KeyValueStorageFactory stora | |
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| log.info("Closing EntryLocationIndex"); | ||
| while (!compacting.compareAndSet(false, true)) { | ||
| // Wait till the locationsDb stops compacting | ||
| log.info("Waiting the locationsDb stops compacting"); | ||
| try { | ||
| Thread.sleep(1000); | ||
| } catch (InterruptedException e) { | ||
| Thread.currentThread().interrupt(); | ||
| throw new IOException(e); | ||
| } | ||
| } | ||
| locationsDb.close(); | ||
| log.info("Closed EntryLocationIndex"); | ||
| } | ||
|
|
||
| public long getLocation(long ledgerId, long entryId) throws IOException { | ||
|
|
@@ -203,15 +218,17 @@ public String getEntryLocationDBPath() { | |
|
|
||
| public void compact() throws IOException { | ||
| try { | ||
| isCompacting = true; | ||
| if (!compacting.compareAndSet(false, true)) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have a question: Is the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. It's the root cause. We should cancel the compacting if we have closed the DB, or we should delay closing the DB if we have already been in compaction status. So here we need an atomic variable to serve as a flag for the DB status. @StevenLuMT |
||
| return; | ||
| } | ||
| locationsDb.compact(); | ||
| } finally { | ||
| isCompacting = false; | ||
| compacting.set(false); | ||
| } | ||
| } | ||
|
|
||
| public boolean isCompacting() { | ||
| return isCompacting; | ||
| return compacting.get(); | ||
| } | ||
|
|
||
| public void removeOffsetFromDeletedLedgers() throws IOException { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This waiting could result in the system being stuck here indefinitely, or it could take an exceptionally long time to get stuck at this step.
Should we add a maximum waiting time, or do something else, such as modifying RocksDB operations?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I dont think so. Closing a DB which in compaction status may triger core dumps or other unexpected error. If closing the DB cost too long time, I think we'd better find out why or
kill -9if we need.The handling approach here is similar to the procedure below:
bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
Lines 770 to 772 in 3a5cf9d