You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/06/05 20:04:48 UTC

[bookkeeper] branch master updated: Append ledgersMap when entrylog is removed from cache.

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 1b5aba2  Append ledgersMap when entrylog is removed from cache.
1b5aba2 is described below

commit 1b5aba204e0e8b3ae85d467abd42e0478bea99df
Author: cguttapalem <cg...@salesforce.com>
AuthorDate: Tue Jun 5 13:04:38 2018 -0700

    Append ledgersMap when entrylog is removed from cache.
    
    Descriptions of the changes in this PR:
    
    In EntryLogManagerForEntryLogPerLedger when ledger-entrylog
    entry is removed from cache, it will be moved to
    rotatedEntryLogs list. Before moving it to rotatedEntryLogs,
    ledgersMap should be appended.
    
    Author: cguttapalem <cg...@salesforce.com>
    
    Reviewers: Jia Zhai <None>, Sijie Guo <si...@apache.org>
    
    This closes #1465 from reddycharan/fixappendledgersmap
---
 .../EntryLogManagerForEntryLogPerLedger.java       |  6 ++
 .../org/apache/bookkeeper/bookie/EntryLogTest.java | 64 +++++++++++++++++++++-
 2 files changed, 69 insertions(+), 1 deletion(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java
index 3cdbb7a..c1ec237 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogManagerForEntryLogPerLedger.java
@@ -186,6 +186,12 @@ class EntryLogManagerForEntryLogPerLedger extends EntryLogManagerBase {
         lock.lock();
         try {
             BufferedLogChannel logChannel = logChannelWithDirInfo.getLogChannel();
+            // Append ledgers map at the end of entry log
+            try {
+                logChannel.appendLedgersMap();
+            } catch (Exception e) {
+                log.error("Got IOException while trying to appendLedgersMap in cacheEntryRemoval callback", e);
+            }
             replicaOfCurrentLogChannels.remove(logChannel.getLogId());
             rotatedLogChannels.add(logChannel);
         } finally {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
index fbfe7c9..65fe4c9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
@@ -59,6 +59,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap;
 import org.apache.commons.io.FileUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -960,7 +961,7 @@ public class EntryLogTest {
             ServerConfiguration servConf) throws IOException {
         File tmpFile = File.createTempFile("entrylog", logid + "");
         tmpFile.deleteOnExit();
-        FileChannel fc = FileChannel.open(tmpFile.toPath());
+        FileChannel fc = new RandomAccessFile(tmpFile, "rw").getChannel();
         EntryLogger.BufferedLogChannel logChannel = new BufferedLogChannel(fc, 10, 10, logid, tmpFile,
                 servConf.getFlushIntervalInBytes());
         return logChannel;
@@ -1089,6 +1090,67 @@ public class EntryLogTest {
         }
     }
 
+    /*
+     * when entrylog for ledger is removed from ledgerIdEntryLogMap, then
+     * ledgermap should be appended to that entrylog, before moving that
+     * entrylog to rotatedlogchannels.
+     */
+    @Test
+    public void testAppendLedgersMapOnCacheRemoval() throws Exception {
+        final int cacheMaximumSize = 5;
+
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setEntryLogFilePreAllocationEnabled(true);
+        conf.setEntryLogPerLedgerEnabled(true);
+        conf.setLedgerDirNames(createAndGetLedgerDirs(1));
+        conf.setMaximumNumberOfActiveEntryLogs(cacheMaximumSize);
+        LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(),
+                new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
+
+        EntryLogger entryLogger = new EntryLogger(conf, ledgerDirsManager);
+        EntryLogManagerForEntryLogPerLedger entryLogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger
+                .getEntryLogManager();
+
+        long ledgerId = 0l;
+        entryLogManager.createNewLog(ledgerId);
+        int entrySize = 200;
+        int numOfEntries = 4;
+        for (int i = 0; i < numOfEntries; i++) {
+            entryLogger.addEntry(ledgerId, generateEntry(ledgerId, i, entrySize));
+        }
+
+        BufferedLogChannel logChannelForledger = entryLogManager.getCurrentLogForLedger(ledgerId);
+        long logIdOfLedger = logChannelForledger.getLogId();
+        /*
+         * do checkpoint to make sure entrylog files are persisted
+         */
+        entryLogger.checkpoint();
+
+        try {
+            entryLogger.extractEntryLogMetadataFromIndex(logIdOfLedger);
+        } catch (IOException ie) {
+            // expected because appendLedgersMap wouldn't have been called
+        }
+
+        /*
+         * create entrylogs for more ledgers, so that ledgerIdEntryLogMap would
+         * reach its limit and remove the oldest entrylog.
+         */
+        for (int i = 1; i <= cacheMaximumSize; i++) {
+            entryLogManager.createNewLog(i);
+        }
+        /*
+         * do checkpoint to make sure entrylog files are persisted
+         */
+        entryLogger.checkpoint();
+
+        EntryLogMetadata entryLogMetadata = entryLogger.extractEntryLogMetadataFromIndex(logIdOfLedger);
+        ConcurrentLongLongHashMap ledgersMap = entryLogMetadata.getLedgersMap();
+        Assert.assertEquals("There should be only one entry in entryLogMetadata", 1, ledgersMap.size());
+        Assert.assertTrue("Usage should be 1", Double.compare(1.0, entryLogMetadata.getUsage()) == 0);
+        Assert.assertEquals("Total size of entries", (entrySize + 4) * numOfEntries, ledgersMap.get(ledgerId));
+    }
+
     /**
      * test EntryLogManager.EntryLogManagerForEntryLogPerLedger doesn't removes
      * the ledger from its cache map if ledger's corresponding state is accessed

-- 
To stop receiving notification emails like this one, please contact
sijie@apache.org.