You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/06/05 20:04:49 UTC

[GitHub] sijie closed pull request #1465: Append ledgersMap when entrylog is removed from cache.

sijie closed pull request #1465: Append ledgersMap when entrylog is removed from cache.
URL: https://github.com/apache/bookkeeper/pull/1465
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 3cdbb7a2a..c1ec23740 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 @@ private void onCacheEntryRemoval(RemovalNotification<Long, EntryLogAndLockTuple>
         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 f5e73a167..f231b4510 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.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;
@@ -959,7 +960,7 @@ public void testEntryLogManagerInterfaceForEntryLogPerLedger() throws Exception
             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;
@@ -1088,6 +1089,67 @@ public void testCacheMaximumSizeEvictionPolicy() throws Exception {
         }
     }
 
+    /*
+     * 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


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services