This is an automated email from the ASF dual-hosted git repository.
sijie pushed a commit to branch branch-4.10
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/branch-4.10 by this push:
new f79c760 Add Journal PageCache flush interval control when
journalSyncData is disabled
f79c760 is described below
commit f79c760d1fd2fa7f7f783519b7d8910043182546
Author: hangc0276 <[email protected]>
AuthorDate: Thu Mar 19 07:08:25 2020 +0800
Add Journal PageCache flush interval control when journalSyncData is
disabled
Descriptions of the changes in this PR:
### Motivation
There was a weird behavior observed: when journalSyncData is disabled, it
will trigger callbacks when the data is flushed to filesystem(PageCache) but
still enqueue a fsync request to force write queue to fsync PageCache data to
disk frequently. Though the fsync thread dequeue fsync request and execute
fsync operation asynchronously, it still cause disk io util increase when fsync
frequency high enough. when disk io util increase, especially reach 100%, it
will block new journal file [...]
### Changes
when journalSyncData is disabled, we introduce
`journalPageCacheFlushIntervalMSec` to control journal PageCache flush
frequency to reduce disk io util, default value is 1000ms
Reviewers: Enrico Olivelli <[email protected]>, Jia Zhai
<[email protected]>, Sijie Guo <None>
This closes #2287 from hangc0276/bugfix
(cherry picked from commit 8201de86a6e04164efa4fabbf91d4c4a67bbd1fe)
Signed-off-by: Sijie Guo <[email protected]>
---
.../java/org/apache/bookkeeper/bookie/Journal.java | 22 +-
.../bookkeeper/conf/ServerConfiguration.java | 22 ++
.../bookie/BookieJournalPageCacheFlushTest.java | 297 +++++++++++++++++++++
conf/bk_server.conf | 3 +
4 files changed, 342 insertions(+), 2 deletions(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
index 5d09e2f..4d6d2e8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
@@ -606,6 +606,8 @@ public class Journal extends BookieCriticalThread
implements CheckpointSource {
private final boolean removePagesFromCache;
private final int journalFormatVersionToWrite;
private final int journalAlignmentSize;
+ // control PageCache flush interval when syncData disabled to reduce disk
io util
+ private final long journalPageCacheFlushIntervalMSec;
// Should data be fsynced on disk before triggering the callback
private final boolean syncData;
@@ -666,6 +668,7 @@ public class Journal extends BookieCriticalThread
implements CheckpointSource {
this.bufferedEntriesThreshold =
conf.getJournalBufferedEntriesThreshold();
this.journalFormatVersionToWrite =
conf.getJournalFormatVersionToWrite();
this.journalAlignmentSize = conf.getJournalAlignmentSize();
+ this.journalPageCacheFlushIntervalMSec =
conf.getJournalPageCacheFlushIntervalMSec();
if (conf.getNumJournalCallbackThreads() > 0) {
this.cbThreadPool =
Executors.newFixedThreadPool(conf.getNumJournalCallbackThreads(),
new
DefaultThreadFactory("bookie-journal-callback"));
@@ -933,6 +936,7 @@ public class Journal extends BookieCriticalThread
implements CheckpointSource {
boolean groupWhenTimeout = false;
long dequeueStartTime = 0L;
+ long lastFlushTimeMs = System.currentTimeMillis();
QueueEntry qe = null;
while (true) {
@@ -1052,8 +1056,22 @@ public class Journal extends BookieCriticalThread
implements CheckpointSource {
boolean shouldRolloverJournal = (lastFlushPosition
> maxJournalSize);
// Trigger data sync to disk in the "Force-Write"
thread.
-
forceWriteRequests.put(createForceWriteRequest(logFile, logId,
lastFlushPosition,
-
toFlush, shouldRolloverJournal, false));
+ // Trigger data sync to disk has three situations:
+ // 1. journalSyncData enabled, usually for SSD
used as journal storage
+ // 2. shouldRolloverJournal is true, that is the
journal file reaches maxJournalSize
+ // 3. if journalSyncData disabled and
shouldRolloverJournal is false, we can use
+ // journalPageCacheFlushIntervalMSec to control
sync frequency, preventing disk
+ // synchronize frequently, which will increase
disk io util.
+ // when flush interval reaches
journalPageCacheFlushIntervalMSec (default: 1s),
+ // it will trigger data sync to disk
+ if (syncData
+ || shouldRolloverJournal
+ || (System.currentTimeMillis() -
lastFlushTimeMs
+ >= journalPageCacheFlushIntervalMSec)) {
+
forceWriteRequests.put(createForceWriteRequest(logFile, logId,
lastFlushPosition,
+ toFlush, shouldRolloverJournal,
false));
+ lastFlushTimeMs = System.currentTimeMillis();
+ }
toFlush = entryListRecycler.newInstance();
numEntriesToFlush = 0;
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
index cba673b..ab49c71 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
@@ -135,6 +135,7 @@ public class ServerConfiguration extends
AbstractConfiguration<ServerConfigurati
protected static final String NUM_JOURNAL_CALLBACK_THREADS =
"numJournalCallbackThreads";
protected static final String JOURNAL_FORMAT_VERSION_TO_WRITE =
"journalFormatVersionToWrite";
protected static final String JOURNAL_QUEUE_SIZE = "journalQueueSize";
+ protected static final String JOURNAL_PAGECACHE_FLUSH_INTERVAL_MSEC =
"journalPageCacheFlushIntervalMSec";
// backpressure control
protected static final String MAX_ADDS_IN_PROGRESS_LIMIT =
"maxAddsInProgressLimit";
protected static final String MAX_READS_IN_PROGRESS_LIMIT =
"maxReadsInProgressLimit";
@@ -809,6 +810,27 @@ public class ServerConfiguration extends
AbstractConfiguration<ServerConfigurati
}
/**
+ * Set PageCache flush interval in second.
+ *
+ * @Param journalPageCacheFlushInterval
+ * journal pageCache flush interval when journalSyncData closed
+ * @return server configuration.
+ */
+ public ServerConfiguration setJournalPageCacheFlushIntervalMSec(long
journalPageCacheFlushIntervalMSec) {
+ this.setProperty(JOURNAL_PAGECACHE_FLUSH_INTERVAL_MSEC,
journalPageCacheFlushIntervalMSec);
+ return this;
+ }
+
+ /**
+ * Get journal pageCache flush interval.
+ *
+ * @return journal pageCache flush interval.
+ */
+ public long getJournalPageCacheFlushIntervalMSec() {
+ return this.getLong(JOURNAL_PAGECACHE_FLUSH_INTERVAL_MSEC, 1000);
+ }
+
+ /**
* Get max number of adds in progress. 0 == unlimited.
*
* @return Max number of adds in progress.
diff --git
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java
new file mode 100644
index 0000000..34d80f5
--- /dev/null
+++
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieJournalPageCacheFlushTest.java
@@ -0,0 +1,297 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.bookie;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+
+import java.io.File;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Journal.ForceWriteRequest;
+import org.apache.bookkeeper.bookie.Journal.LastLogMark;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.conf.TestBKConfiguration;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.powermock.reflect.Whitebox;
+
+/**
+ * Test the bookie journal PageCache flush interval.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({JournalChannel.class, Journal.class})
+@Slf4j
+public class BookieJournalPageCacheFlushTest {
+
+ private static final ByteBuf DATA = Unpooled.wrappedBuffer(new byte[]{});
+
+ @Rule
+ public TemporaryFolder tempDir = new TemporaryFolder();
+
+ @SuppressWarnings("unchecked")
+ private LinkedBlockingQueue<ForceWriteRequest>
enableForceWriteThreadSuspension(
+ CountDownLatch forceWriteThreadSuspendedLatch,
+ Journal journal) throws InterruptedException {
+ LinkedBlockingQueue<ForceWriteRequest> supportQueue = new
LinkedBlockingQueue<>();
+ BlockingQueue<ForceWriteRequest> forceWriteRequests =
mock(BlockingQueue.class);
+ doAnswer((Answer) (InvocationOnMock iom) -> {
+ supportQueue.put(iom.getArgument(0));
+ return null;
+ }).when(forceWriteRequests).put(any(ForceWriteRequest.class));
+ when(forceWriteRequests.take()).thenAnswer(i -> {
+ // suspend the force write thread
+ forceWriteThreadSuspendedLatch.await();
+ return supportQueue.take();
+ });
+ Whitebox.setInternalState(journal, "forceWriteRequests",
forceWriteRequests);
+ return supportQueue;
+ }
+
+ @Test
+ public void testAckAfterSyncPageCacheFlush() throws Exception {
+ File journalDir = tempDir.newFolder();
+ Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+
+ ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+ .setJournalDirName(journalDir.getPath())
+ .setMetadataServiceUri(null)
+ .setJournalAdaptiveGroupWrites(false)
+ .setJournalSyncData(true)
+ .setJournalPageCacheFlushIntervalMSec(5000);
+
+ JournalChannel jc = spy(new JournalChannel(journalDir, 1));
+ whenNew(JournalChannel.class).withAnyArguments().thenReturn(jc);
+
+ LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class);
+ Journal journal = new Journal(0, journalDir, conf, ledgerDirsManager);
+
+ CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1);
+ LinkedBlockingQueue<ForceWriteRequest> supportQueue =
+
enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal);
+ journal.start();
+
+ LogMark lastLogMarkBeforeWrite =
journal.getLastLogMark().markLog().getCurMark();
+ CountDownLatch latch = new CountDownLatch(1);
+ long ledgerId = 1;
+ long entryId = 0;
+ long startTime = System.currentTimeMillis();
+ journal.logAddEntry(ledgerId, entryId, DATA, false /* ackBeforeSync
*/, new WriteCallback() {
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId,
BookieSocketAddress addr, Object ctx) {
+ latch.countDown();
+ }
+ }, null);
+
+ while (supportQueue.isEmpty()) {
+ Thread.sleep(100);
+ }
+
+ // forceWriteRequest insert into forceWriteRequestQueue not effected
by journalPageCacheFlushInterval
+ assertTrue(System.currentTimeMillis() - startTime < 5000);
+
+ assertEquals(1, latch.getCount());
+ assertEquals(1, supportQueue.size());
+
+ // in constructor of JournalChannel we are calling forceWrite(true)
but it is not tracked by PowerMock
+ // because the 'spy' is applied only on return from the constructor
+ verify(jc, times(0)).forceWrite(true);
+
+ // should not call forceWrite
+ verify(jc, times(0)).forceWrite(false);
+
+ // let ForceWriteThread work
+ forceWriteThreadSuspendedLatch.countDown();
+ // callback should complete now
+ assertTrue(latch.await(20, TimeUnit.SECONDS));
+
+ verify(jc, times(1)).forceWrite(false);
+ assertEquals(0, supportQueue.size());
+
+ // verify that log marker advanced
+ LastLogMark lastLogMarkAfterForceWrite = journal.getLastLogMark();
+
assertTrue(lastLogMarkAfterForceWrite.getCurMark().compare(lastLogMarkBeforeWrite)
> 0);
+
+ journal.shutdown();
+ }
+
+ @Test
+ public void testAckBeforeSyncPageCacheFlush() throws Exception {
+ File journalDir = tempDir.newFolder();
+ Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+
+ ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+ .setJournalDirName(journalDir.getPath())
+ .setMetadataServiceUri(null)
+ .setJournalAdaptiveGroupWrites(false)
+ .setJournalSyncData(true)
+ .setJournalPageCacheFlushIntervalMSec(5000);
+
+ JournalChannel jc = spy(new JournalChannel(journalDir, 1));
+ whenNew(JournalChannel.class).withAnyArguments().thenReturn(jc);
+
+ LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class);
+ Journal journal = new Journal(0, journalDir, conf, ledgerDirsManager);
+
+ CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1);
+ LinkedBlockingQueue<ForceWriteRequest> supportQueue =
+
enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal);
+ journal.start();
+
+ LogMark lastLogMarkBeforeWrite =
journal.getLastLogMark().markLog().getCurMark();
+ CountDownLatch latch = new CountDownLatch(1);
+ long ledgerId = 1;
+ long entryId = 0;
+ long startTime = System.currentTimeMillis();
+ journal.logAddEntry(ledgerId, entryId, DATA, true /* ackBeforeSync */,
new WriteCallback() {
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId,
BookieSocketAddress addr, Object ctx) {
+ latch.countDown();
+ }
+ }, null);
+
+ while (supportQueue.isEmpty()) {
+ Thread.sleep(100);
+ }
+
+ // forceWriteRequest insert into forceWriteRequestQueue not effected
by journalPageCacheFlushInterval
+ assertTrue(System.currentTimeMillis() - startTime < 5000);
+ assertEquals(1, supportQueue.size());
+
+ // callback should completed now
+ assertTrue(latch.await(20, TimeUnit.SECONDS));
+
+ // in constructor of JournalChannel we are calling forceWrite(true)
but it is not tracked by PowerMock
+ // because the 'spy' is applied only on return from the constructor
+ verify(jc, times(0)).forceWrite(true);
+
+ // we are never calling foreWrite
+ verify(jc, times(0)).forceWrite(false);
+
+ // verify that log marker did not advance
+ LastLogMark lastLogMarkAfterForceWrite = journal.getLastLogMark();
+ assertEquals(0,
lastLogMarkAfterForceWrite.getCurMark().compare(lastLogMarkBeforeWrite));
+
+ // let the forceWriteThread exit
+ forceWriteThreadSuspendedLatch.countDown();
+
+ journal.shutdown();
+ }
+
+ @Test
+ public void testAckBeforeUnSyncPageCacheFlush() throws Exception {
+ File journalDir = tempDir.newFolder();
+ Bookie.checkDirectoryStructure(Bookie.getCurrentDirectory(journalDir));
+
+ ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+ .setJournalDirName(journalDir.getPath())
+ .setMetadataServiceUri(null)
+ .setJournalAdaptiveGroupWrites(false)
+ .setJournalSyncData(false)
+ .setJournalPageCacheFlushIntervalMSec(5000);
+
+ JournalChannel jc = spy(new JournalChannel(journalDir, 1));
+ whenNew(JournalChannel.class).withAnyArguments().thenReturn(jc);
+
+ LedgerDirsManager ledgerDirsManager = mock(LedgerDirsManager.class);
+ Journal journal = new Journal(0, journalDir, conf, ledgerDirsManager);
+
+ CountDownLatch forceWriteThreadSuspendedLatch = new CountDownLatch(1);
+ LinkedBlockingQueue<ForceWriteRequest> supportQueue =
+
enableForceWriteThreadSuspension(forceWriteThreadSuspendedLatch, journal);
+ journal.start();
+
+ CountDownLatch latch = new CountDownLatch(2);
+ long ledgerId = 1;
+ long entryId = 0;
+ LogMark lastLogMarkBeforeWrite =
journal.getLastLogMark().markLog().getCurMark();
+ journal.logAddEntry(ledgerId, entryId, DATA, true, new WriteCallback()
{
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId,
BookieSocketAddress addr, Object ctx) {
+ latch.countDown();
+ }
+ }, null);
+
+ // the forceWriteRequest should not generated because of
journalPageCacheFlushIntervalMSec control
+ assertEquals(0, supportQueue.size());
+
+ // wait journalPageCacheFlushIntervalMsec timeout
+ Thread.sleep(10000);
+
+ // add an entry to journal, wake up journal main thread which is
blocked on queue.take()
+ journal.logAddEntry(ledgerId, entryId + 1, DATA, true, new
WriteCallback() {
+ @Override
+ public void writeComplete(int rc, long ledgerId, long entryId,
BookieSocketAddress addr, Object ctx) {
+ latch.countDown();
+ }
+ }, null);
+
+ // wait forceWriteRequest generated
+ while (supportQueue.isEmpty()) {
+ Thread.sleep(100);
+ }
+
+ // only one forceWriteRequest inserted into forceWriteRequestQueue
+ assertEquals(1, supportQueue.size());
+
+ // callback should completed now
+ assertTrue(latch.await(20, TimeUnit.SECONDS));
+
+ // in constructor of JournalChannel we are calling forceWrite(true)
but it is not tracked by PowerMock
+ // because the 'spy' is applied only on return from the constructor
+ verify(jc, times(0)).forceWrite(true);
+
+ // we are never calling foreWrite
+ verify(jc, times(0)).forceWrite(false);
+
+ // verify that log marker did not advance
+ LastLogMark lastLogMarkAfterForceWrite = journal.getLastLogMark();
+ assertEquals(0,
lastLogMarkAfterForceWrite.getCurMark().compare(lastLogMarkBeforeWrite));
+
+ // let the forceWriteThread exit
+ forceWriteThreadSuspendedLatch.countDown();
+
+ journal.shutdown();
+ }
+}
diff --git a/conf/bk_server.conf b/conf/bk_server.conf
index 4da030d..0412005 100755
--- a/conf/bk_server.conf
+++ b/conf/bk_server.conf
@@ -343,6 +343,9 @@ journalDirectories=/tmp/bk-txn
# Set the size of the journal queue.
# journalQueueSize=10000
+# Set PageCache flush interval (millisecond) when journalSyncData disabled
+# journalPageCacheFlushIntervalMSec = 1000
+
#############################################################################
## Ledger storage settings
#############################################################################