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/03/24 21:41:49 UTC

[bookkeeper] branch master updated: Improve FileInfoBackingCache

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 1bd2c0e  Improve FileInfoBackingCache
1bd2c0e is described below

commit 1bd2c0eef3af8810e70116633b95c4133183d873
Author: Sijie Guo <si...@apache.org>
AuthorDate: Sat Mar 24 14:41:42 2018 -0700

    Improve FileInfoBackingCache
    
    Descriptions of the changes in this PR:
    
    There are a couple of issues noticed in FileInfoBackingCache:
    
    1) There is a race condition in loadFileInfo between get-check and put. If concurrent loading happens, there might be a FileInfo loaded into the map after get-check. This can cause incorrect reference count on FileInfo.
    
    2) FileLoader is doing I/O operation which happens under a giant write lock.
    
    3) assert is typically not recommended since it is disabled at production runtime typically.
    
    *Changes*
    
    - Check whether fileinfo exists or not after getting write lock and before put
    - Move any I/O operations out of write lock
    - release the new FileInfo if concurrent puts happen
    - remove the usage of assert
    
    Beside that, switch to use ConcurrentLongHashMap to avoid boxing and unboxing.
    
    Related Issues:
    
    #913 #832
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Ivan Kelly <iv...@apache.org>
    
    This closes #1284 from sijie/improve_fileinfo_backing_cache
---
 .../bookkeeper/bookie/FileInfoBackingCache.java    | 51 ++++++++++++++++------
 1 file changed, 37 insertions(+), 14 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java
index e4cb183..31fd077 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfoBackingCache.java
@@ -22,24 +22,35 @@ package org.apache.bookkeeper.bookie;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+import java.io.UncheckedIOException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap;
 
 @Slf4j
 class FileInfoBackingCache {
     static final int DEAD_REF = -0xdead;
 
     final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-    final ConcurrentHashMap<Long, CachedFileInfo> fileInfos = new ConcurrentHashMap<>();
+    final ConcurrentLongHashMap<CachedFileInfo> fileInfos = new ConcurrentLongHashMap<>();
     final FileLoader fileLoader;
 
     FileInfoBackingCache(FileLoader fileLoader) {
         this.fileLoader = fileLoader;
     }
 
+    /**
+     * This method should be under `lock` of FileInfoBackingCache.
+     */
+    private static CachedFileInfo tryRetainFileInfo(CachedFileInfo fi) throws IOException {
+        boolean retained = fi.tryRetain();
+        if (!retained) {
+            throw new IOException("FileInfo " + fi + " is already marked dead");
+        }
+        return fi;
+    }
+
     CachedFileInfo loadFileInfo(long ledgerId, byte[] masterKey) throws IOException {
         lock.readLock().lock();
         try {
@@ -52,25 +63,29 @@ class FileInfoBackingCache {
                 // have been able to get a reference to it here.
                 // The caller of loadFileInfo owns the refence, and is
                 // responsible for calling the corresponding #release().
-                boolean retained = fi.tryRetain();
-                assert(retained);
-                return fi;
+                return tryRetainFileInfo(fi);
             }
         } finally {
             lock.readLock().unlock();
         }
 
+        File backingFile = fileLoader.load(ledgerId, masterKey != null);
+        CachedFileInfo newFi = new CachedFileInfo(ledgerId, backingFile, masterKey);
+
         // else FileInfo not found, create it under write lock
         lock.writeLock().lock();
         try {
-            File backingFile = fileLoader.load(ledgerId, masterKey != null);
-            CachedFileInfo fi = new CachedFileInfo(ledgerId, backingFile, masterKey);
-            fileInfos.put(ledgerId, fi);
+            CachedFileInfo fi = fileInfos.get(ledgerId);
+            if (fi != null) {
+                // someone is already putting a fileinfo here, so use the existing one and recycle the new one
+                newFi.recycle();
+            } else {
+                fileInfos.put(ledgerId, newFi);
+                fi = newFi;
+            }
 
             // see comment above for why we assert
-            boolean retained = fi.tryRetain();
-            assert(retained);
-            return fi;
+            return tryRetainFileInfo(fi);
         } finally {
             lock.writeLock().unlock();
         }
@@ -92,8 +107,16 @@ class FileInfoBackingCache {
     }
 
     void closeAllWithoutFlushing() throws IOException {
-        for (Map.Entry<Long, CachedFileInfo> entry : fileInfos.entrySet()) {
-            entry.getValue().close(false);
+        try {
+            fileInfos.forEach((key, fileInfo) -> {
+                try {
+                    fileInfo.close(false);
+                } catch (IOException e) {
+                    throw new UncheckedIOException(e);
+                }
+            });
+        } catch (UncheckedIOException uioe) {
+            throw uioe.getCause();
         }
     }
 

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