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/01/03 00:54:24 UTC

[GitHub] sijie commented on a change in pull request #832: Issue 620: Close the fileChannels for read when they are idle

sijie commented on a change in pull request #832: Issue 620: Close the fileChannels for read when they are idle
URL: https://github.com/apache/bookkeeper/pull/832#discussion_r159347434
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
 ##########
 @@ -315,6 +316,104 @@ public void testRecoverFromLedgersMapOnV0EntryLog() throws Exception {
         assertEquals(120, meta.getRemainingSize());
     }
 
+    /**
+     * Test Cache for logid2Channel and concurrentMap for logid2FileChannel work correctly.
+     * Note that, when an entryLogger is initialized, the entry log id will increase one.
+     * when the preallocation is enabled, a new entrylogger will cost 2 logId.
+     */
+    @Test
+    public void testCacheInEntryLog() throws Exception {
+        File tmpDir = createTempDir("bkTest", ".dir");
+        File curDir = Bookie.getCurrentDirectory(tmpDir);
+        Bookie.checkDirectoryStructure(curDir);
+
+        int gcWaitTime = 1000;
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setGcWaitTime(gcWaitTime);
+        conf.setLedgerDirNames(new String[] {tmpDir.toString()});
+        //since last access, expire after 1s
+        conf.setExpireReadChannelCache(1000);
+        conf.setEntryLogFilePreAllocationEnabled(false);
+        // below one will cost logId 0
+        Bookie bookie = new Bookie(conf);
+        // create some entries
+        int numLogs = 4;
+        int numEntries = 10;
+        long[][] positions = new long[numLogs][];
+        for (int i = 0; i < numLogs; i++) {
+            positions[i] = new long[numEntries];
+            EntryLogger logger = new EntryLogger(conf,
+                    bookie.getLedgerDirsManager());
+            for (int j = 0; j < numEntries; j++) {
+                positions[i][j] = logger.addEntry(i, generateEntry(i, j).nioBuffer());
+            }
+            logger.flush();
+            LOG.info("log id is {}, LeastUnflushedLogId is {} ", logger.getCurrentLogId(),
+                    logger.getLeastUnflushedLogId());
+        }
+
+        for (int i = 1; i < numLogs + 1; i++) {
+            File logFile = new File(curDir, Long.toHexString(i) + ".log");
+            assertTrue(logFile.exists());
+        }
+
+        // create some read for the entry log
+        EntryLogger logger = ((InterleavedLedgerStorage) bookie.ledgerStorage).entryLogger;
+        ThreadLocal<Cache<Long, BufferedReadChannel>>  cacheThreadLocal = logger.getLogid2Channel();
+        ConcurrentMap<Long, EntryLogger.ReferenceCountedFileChannel> logid2FileChannel = logger.getLogid2FileChannel();
+        for (int j = 0; j < numEntries; j++) {
+            logger.readEntry(0, j, positions[0][j]);
+        }
+        LOG.info("cache size is {}, content is {}", cacheThreadLocal.get().size(),
+                cacheThreadLocal.get().asMap().toString());
+        // the cache has readChannel for 1.log
+        assertNotNull(cacheThreadLocal.get().getIfPresent(1L));
+        for (int j = 0; j < numEntries; j++) {
+            logger.readEntry(1, j, positions[1][j]);
+        }
+        LOG.info("cache size is {}, content is {}", cacheThreadLocal.get().size(),
+                cacheThreadLocal.get().asMap().toString());
+        // the cache has readChannel for 2.log
+        assertNotNull(cacheThreadLocal.get().getIfPresent(2L));
+        // expire time
+        Thread.sleep(1000);
 
 Review comment:
   @ArvinDevel : guava cache has a mock ticker that allows you to advance time to trigger expriation.

----------------------------------------------------------------
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