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 15:57:37 UTC

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

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

 ##########
 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:
   @ivankelly do your think this can be a blocker for 4.6 release ?
   @jiazhai 

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