Skip to content

Commit

Permalink
Fix
Browse files Browse the repository at this point in the history
  • Loading branch information
nodece committed Feb 21, 2024
1 parent 769b3dd commit e8b41ad
Show file tree
Hide file tree
Showing 2 changed files with 45 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@
/**
* Implements a bookie.
*/
public class BookieImpl extends BookieCriticalThread implements Bookie {
public class BookieImpl implements Bookie {

private static final Logger LOG = LoggerFactory.getLogger(Bookie.class);

Expand Down Expand Up @@ -119,6 +119,8 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {

protected StateManager stateManager;

private BookieCriticalThread bookieThread;

// Expose Stats
final StatsLogger statsLogger;
private final BookieStats bookieStats;
Expand Down Expand Up @@ -390,7 +392,6 @@ public BookieImpl(ServerConfiguration conf,
ByteBufAllocator allocator,
Supplier<BookieServiceInfo> bookieServiceInfoProvider)
throws IOException, InterruptedException, BookieException {
super("Bookie-" + conf.getBookiePort());
this.bookieServiceInfoProvider = bookieServiceInfoProvider;
this.statsLogger = statsLogger;
this.conf = conf;
Expand Down Expand Up @@ -656,7 +657,9 @@ private void replay(Journal journal, JournalScanner scanner) throws IOException

@Override
public synchronized void start() {
setDaemon(true);
bookieThread = new BookieCriticalThread(() -> run(), "Bookie-" + conf.getBookiePort());
bookieThread.setDaemon(true);

ThreadRegistry.register("BookieThread", 0);
if (LOG.isDebugEnabled()) {
LOG.debug("I'm starting a bookie with journal directories {}",
Expand Down Expand Up @@ -717,7 +720,7 @@ public synchronized void start() {
syncThread.start();

// start bookie thread
super.start();
bookieThread.start();

// After successful bookie startup, register listener for disk
// error/full notifications.
Expand All @@ -741,6 +744,20 @@ public synchronized void start() {
}
}

@Override
public void join() throws InterruptedException {
if (bookieThread != null) {
bookieThread.join();
}
}

public boolean isAlive() {
if (bookieThread == null) {
return false;
}
return bookieThread.isAlive();
}

/*
* Get the DiskFailure listener for the bookie
*/
Expand Down Expand Up @@ -824,7 +841,6 @@ public boolean isRunning() {
return stateManager.isRunning();
}

@Override
public void run() {
// start journals
for (Journal journal: journals) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException;
import org.apache.bookkeeper.bookie.stats.JournalStats;
import org.apache.bookkeeper.common.collections.BatchedArrayBlockingQueue;
Expand All @@ -66,13 +67,15 @@
/**
* Provide journal related management.
*/
public class Journal extends BookieCriticalThread implements CheckpointSource {
public class Journal implements CheckpointSource {

private static final Logger LOG = LoggerFactory.getLogger(Journal.class);

private static final RecyclableArrayList.Recycler<QueueEntry> entryListRecycler =
new RecyclableArrayList.Recycler<QueueEntry>();

private BookieCriticalThread thread;

/**
* Filter to pickup journals.
*/
Expand Down Expand Up @@ -461,13 +464,13 @@ private class ForceWriteThread extends BookieCriticalThread {
volatile boolean running = true;
// This holds the queue entries that should be notified after a
// successful force write
Thread threadToNotifyOnEx;
Consumer<Void> threadToNotifyOnEx;

// should we group force writes
private final boolean enableGroupForceWrites;
private final Counter forceWriteThreadTime;

public ForceWriteThread(Thread threadToNotifyOnEx,
public ForceWriteThread(Consumer<Void> threadToNotifyOnEx,
boolean enableGroupForceWrites,
StatsLogger statsLogger) {
super("ForceWriteThread");
Expand Down Expand Up @@ -531,7 +534,7 @@ public void run() {
// Regardless of what caused us to exit, we should notify the
// the parent thread as it should either exit or be in the process
// of exiting else we will have write requests hang
threadToNotifyOnEx.interrupt();
threadToNotifyOnEx.accept(null);
}

private void syncJournal(ForceWriteRequest lastRequest) throws IOException {
Expand Down Expand Up @@ -652,8 +655,6 @@ public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf

public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf,
LedgerDirsManager ledgerDirsManager, StatsLogger statsLogger, ByteBufAllocator allocator) {
super(journalThreadName + "-" + conf.getBookiePort());
this.setPriority(Thread.MAX_PRIORITY);
this.allocator = allocator;

StatsLogger journalStatsLogger = statsLogger.scopeLabel("journalIndex", String.valueOf(journalIndex));
Expand All @@ -678,7 +679,7 @@ public Journal(int journalIndex, File journalDirectory, ServerConfiguration conf
this.journalWriteBufferSize = conf.getJournalWriteBufferSizeKB() * KB;
this.syncData = conf.getJournalSyncData();
this.maxBackupJournals = conf.getMaxBackupJournals();
this.forceWriteThread = new ForceWriteThread(this, conf.getJournalAdaptiveGroupWrites(),
this.forceWriteThread = new ForceWriteThread((__) -> this.interrupt(), conf.getJournalAdaptiveGroupWrites(),
journalStatsLogger);
this.maxGroupWaitInNanos = TimeUnit.MILLISECONDS.toNanos(conf.getJournalMaxGroupWaitMSec());
this.bufferedWritesThreshold = conf.getJournalBufferedWritesThreshold();
Expand Down Expand Up @@ -956,7 +957,6 @@ journalFormatVersionToWrite, getBufferedChannelBuilder(),
* </p>
* @see org.apache.bookkeeper.bookie.SyncThread
*/
@Override
public void run() {
LOG.info("Starting journal on {}", journalDirectory);
ThreadRegistry.register(journalThreadName, 0);
Expand Down Expand Up @@ -1256,7 +1256,7 @@ public synchronized void shutdown() {

running = false;
this.interrupt();
this.join();
this.joinThread();
LOG.info("Finished Shutting down Journal thread");
} catch (IOException | InterruptedException ie) {
Thread.currentThread().interrupt();
Expand Down Expand Up @@ -1284,7 +1284,21 @@ private static int fullRead(JournalChannel fc, ByteBuffer bb) throws IOException
*/
@VisibleForTesting
public void joinThread() throws InterruptedException {
join();
if (thread != null) {
thread.join();
}
}

public void interrupt() {
if (thread != null) {
thread.interrupt();
}
}

public synchronized void start() {
thread = new BookieCriticalThread(() -> run(), journalThreadName + "-" + conf.getBookiePort());
thread.setPriority(Thread.MAX_PRIORITY);
thread.start();
}

long getMemoryUsage() {
Expand Down

0 comments on commit e8b41ad

Please sign in to comment.