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 2017/12/13 14:39:06 UTC

[GitHub] ivankelly commented on a change in pull request #513: BOOKKEEPER-1106: Introduce write FileInfo cache and read FileInfo cache

ivankelly commented on a change in pull request #513: BOOKKEEPER-1106: Introduce write FileInfo cache and read FileInfo cache
URL: https://github.com/apache/bookkeeper/pull/513#discussion_r156676238
 
 

 ##########
 File path: bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
 ##########
 @@ -100,71 +118,160 @@ public IndexPersistenceMgr(int pageSize,
         getActiveLedgers();
         ledgerDirsManager.addLedgerDirsListener(getLedgerDirsListener());
 
+        // build the file info cache
+        int concurrencyLevel = Math.max(1, Math.max(conf.getNumAddWorkerThreads(), conf.getNumReadWorkerThreads()));
+        RemovalListener<Long, FileInfo> fileInfoEvictionListener = this::handleLedgerEviction;
+        writeFileInfoCache = buildCache(
+            concurrencyLevel,
+            conf.getFileInfoCacheInitialCapacity(),
+            openFileLimit,
+            conf.getFileInfoMaxIdleTime(),
+            fileInfoEvictionListener);
+        readFileInfoCache = buildCache(
+            concurrencyLevel,
+            2 * conf.getFileInfoCacheInitialCapacity(),
+            2 * openFileLimit,
+            conf.getFileInfoMaxIdleTime(),
+            fileInfoEvictionListener);
+
         // Expose Stats
         evictedLedgersCounter = statsLogger.getCounter(LEDGER_CACHE_NUM_EVICTED_LEDGERS);
-        statsLogger.registerGauge(NUM_OPEN_LEDGERS, new Gauge<Integer>() {
+        pendingGetFileInfoCounter = statsLogger.getCounter(PENDING_GET_FILE_INFO);
+        statsLogger.registerGauge(WRITE_FILE_INFO_CACHE_SIZE, new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return writeFileInfoCache.size();
+            }
+        });
+        statsLogger.registerGauge(READ_FILE_INFO_CACHE_SIZE, new Gauge<Number>() {
             @Override
-            public Integer getDefaultValue() {
+            public Number getDefaultValue() {
                 return 0;
             }
 
             @Override
-            public Integer getSample() {
-                return getNumOpenLedgers();
+            public Number getSample() {
+                return readFileInfoCache.size();
             }
         });
     }
 
-    FileInfo getFileInfo(Long ledger, byte masterKey[]) throws IOException {
-        FileInfo fi = fileInfoCache.get(ledger);
-        if (null == fi) {
-            boolean createdNewFile = false;
-            File lf = null;
-            synchronized (this) {
-                // Check if the index file exists on disk.
-                lf = findIndexFile(ledger);
-                if (null == lf) {
-                    if (null == masterKey) {
-                        throw new Bookie.NoLedgerException(ledger);
+    private static Cache<Long, FileInfo> buildCache(int concurrencyLevel,
+                                            int initialCapacity,
+                                            int maximumSize,
+                                            long expireAfterAccessSeconds,
+                                            RemovalListener<Long, FileInfo> removalListener) {
+        CacheBuilder<Long, FileInfo> builder = CacheBuilder.newBuilder()
+            .concurrencyLevel(concurrencyLevel)
+            .initialCapacity(initialCapacity)
+            .maximumSize(maximumSize)
+            .removalListener(removalListener);
+        if (expireAfterAccessSeconds > 0) {
+            builder.expireAfterAccess(expireAfterAccessSeconds, TimeUnit.SECONDS);
+        }
+        return builder.build();
+    }
+
+    /**
+     * When a ledger is evicted, we need to make sure there's no other thread
+     * trying to get FileInfo for that ledger at the same time when we close
+     * the FileInfo.
+     */
+    private void handleLedgerEviction(RemovalNotification<Long, FileInfo> notification) {
+        FileInfo fileInfo = notification.getValue();
+        Long ledgerId = notification.getKey();
+        if (null == fileInfo || null == notification.getKey()) {
+            return;
+        }
+        if (notification.wasEvicted()) {
+            evictedLedgersCounter.inc();
+            // we need to acquire the write lock in another thread,
+            // otherwise there could be dead lock happening.
+            evictionThreadPool.execute(() -> {
 
 Review comment:
   @yzang @sijie 
   There's a serious bug here (which also causes the flake in IndexPersistenceMgrTest#testEvictFileInfoWhenUnderlyingFileExists).
   
   When evicting from the cache, the FileInfo is removed from the cache, but only flushed asynchronously. If someone tries to read the FileInfo from the cache before the flush runs, they will read stale information, and then this will be written back when that FileInfo eventually gets evicted.
   
   Do either of you recall what the deadlock mentioned in the comment is? LoadingCache notifications happen in the same thread as the get() so when handleLedgerEviction is run we should already have the FileInfoLock.readLock() held.

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