accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ktur...@apache.org
Subject [accumulo] branch 1.8 updated: ACCUMULO-4805 Obtain filemanager lock once when opening files (#380)
Date Fri, 09 Feb 2018 00:32:55 GMT
This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch 1.8
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/1.8 by this push:
     new 86d6a69  ACCUMULO-4805 Obtain filemanager lock once when opening files (#380)
86d6a69 is described below

commit 86d6a69ec8e4e7fe1b187c4a53b8c0a88921ab27
Author: Keith Turner <keith@deenlo.com>
AuthorDate: Thu Feb 8 19:32:52 2018 -0500

    ACCUMULO-4805 Obtain filemanager lock once when opening files (#380)
    
    Before this change the lock on the file manager was obtained F+1 times, where
    F is the number of files.  When there are lots of threads this is a lot of
    contention.  In this change, the lock is only obtained once.
    
    Also in this change the file managers semaphore was switched to non-fair.  The
    fair semaphore is expensive and can cause contention.
---
 .../org/apache/accumulo/tserver/FileManager.java   | 45 ++++++++++------------
 1 file changed, 20 insertions(+), 25 deletions(-)

diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
index a707251..2aaad39 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
@@ -172,7 +172,7 @@ public class FileManager {
     this.dataCache = dataCache;
     this.indexCache = indexCache;
 
-    this.filePermits = new Semaphore(maxOpen, true);
+    this.filePermits = new Semaphore(maxOpen, false);
     this.maxOpen = maxOpen;
     this.fs = fs;
 
@@ -234,7 +234,7 @@ public class FileManager {
     return ofl;
   }
 
-  private void closeReaders(List<FileSKVIterator> filesToClose) {
+  private void closeReaders(Collection<FileSKVIterator> filesToClose) {
     for (FileSKVIterator reader : filesToClose) {
       try {
         reader.close();
@@ -244,7 +244,7 @@ public class FileManager {
     }
   }
 
-  private List<String> takeOpenFiles(Collection<String> files, List<FileSKVIterator>
reservedFiles, Map<FileSKVIterator,String> readersReserved) {
+  private List<String> takeOpenFiles(Collection<String> files, Map<FileSKVIterator,String>
readersReserved) {
     List<String> filesToOpen = new LinkedList<>(files);
     for (Iterator<String> iterator = filesToOpen.iterator(); iterator.hasNext();) {
       String file = iterator.next();
@@ -252,7 +252,6 @@ public class FileManager {
       List<OpenReader> ofl = openFiles.get(file);
       if (ofl != null && ofl.size() > 0) {
         OpenReader openReader = ofl.remove(ofl.size() - 1);
-        reservedFiles.add(openReader.reader);
         readersReserved.put(openReader.reader, file);
         if (ofl.size() == 0) {
           openFiles.remove(file);
@@ -264,23 +263,18 @@ public class FileManager {
     return filesToOpen;
   }
 
-  private synchronized String getReservedReadeFilename(FileSKVIterator reader) {
-    return reservedReaders.get(reader);
-  }
-
-  private List<FileSKVIterator> reserveReaders(KeyExtent tablet, Collection<String>
files, boolean continueOnFailure) throws IOException {
+  private Map<FileSKVIterator,String> reserveReaders(KeyExtent tablet, Collection<String>
files, boolean continueOnFailure) throws IOException {
 
     if (!tablet.isMeta() && files.size() >= maxOpen) {
       throw new IllegalArgumentException("requested files exceeds max open");
     }
 
     if (files.size() == 0) {
-      return Collections.emptyList();
+      return Collections.emptyMap();
     }
 
     List<String> filesToOpen = null;
     List<FileSKVIterator> filesToClose = Collections.emptyList();
-    List<FileSKVIterator> reservedFiles = new ArrayList<>();
     Map<FileSKVIterator,String> readersReserved = new HashMap<>();
 
     if (!tablet.isMeta()) {
@@ -295,7 +289,7 @@ public class FileManager {
     // a synch block
     synchronized (this) {
 
-      filesToOpen = takeOpenFiles(files, reservedFiles, readersReserved);
+      filesToOpen = takeOpenFiles(files, readersReserved);
 
       int numOpen = countReaders(openFiles);
 
@@ -318,7 +312,6 @@ public class FileManager {
         // log.debug("Opening "+file + " path " + path);
         FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder().forFile(path.toString(),
ns, ns.getConf())
             .withTableConfiguration(context.getServerConfigurationFactory().getTableConfiguration(tablet)).withBlockCache(dataCache,
indexCache).build();
-        reservedFiles.add(reader);
         readersReserved.put(reader, file);
       } catch (Exception e) {
 
@@ -332,7 +325,7 @@ public class FileManager {
           log.warn("Failed to open file {} {}  continuing...", file, e.getMessage());
         } else {
           // close whatever files were opened
-          closeReaders(reservedFiles);
+          closeReaders(readersReserved.keySet());
 
           if (!tablet.isMeta()) {
             filePermits.release(files.size());
@@ -349,7 +342,7 @@ public class FileManager {
       reservedReaders.putAll(readersReserved);
     }
 
-    return reservedFiles;
+    return readersReserved;
   }
 
   private void releaseReaders(KeyExtent tablet, List<FileSKVIterator> readers, boolean
sawIOException) {
@@ -481,14 +474,14 @@ public class FileManager {
       }
     }
 
-    private List<FileSKVIterator> openFileRefs(Collection<FileRef> files) throws
TooManyFilesException, IOException {
+    private Map<FileSKVIterator,String> openFileRefs(Collection<FileRef> files)
throws TooManyFilesException, IOException {
       List<String> strings = new ArrayList<>(files.size());
       for (FileRef ref : files)
         strings.add(ref.path().toString());
       return openFiles(strings);
     }
 
-    private List<FileSKVIterator> openFiles(Collection<String> files) throws
TooManyFilesException, IOException {
+    private Map<FileSKVIterator,String> openFiles(Collection<String> files) throws
TooManyFilesException, IOException {
       // one tablet can not open more than maxOpen files, otherwise it could get stuck
       // forever waiting on itself to release files
 
@@ -497,16 +490,16 @@ public class FileManager {
             + " files.size()=" + files.size() + " maxOpen=" + maxOpen + " tablet = " + tablet);
       }
 
-      List<FileSKVIterator> newlyReservedReaders = reserveReaders(tablet, files, continueOnFailure);
+      Map<FileSKVIterator,String> newlyReservedReaders = reserveReaders(tablet, files,
continueOnFailure);
 
-      tabletReservedReaders.addAll(newlyReservedReaders);
+      tabletReservedReaders.addAll(newlyReservedReaders.keySet());
       return newlyReservedReaders;
     }
 
     public synchronized List<InterruptibleIterator> openFiles(Map<FileRef,DataFileValue>
files, boolean detachable, SamplerConfigurationImpl samplerConfig)
         throws IOException {
 
-      List<FileSKVIterator> newlyReservedReaders = openFileRefs(files.keySet());
+      Map<FileSKVIterator,String> newlyReservedReaders = openFileRefs(files.keySet());
 
       ArrayList<InterruptibleIterator> iters = new ArrayList<>();
 
@@ -518,8 +511,9 @@ public class FileManager {
         }
       }
 
-      for (FileSKVIterator reader : newlyReservedReaders) {
-        String filename = getReservedReadeFilename(reader);
+      for (Entry<FileSKVIterator,String> entry : newlyReservedReaders.entrySet()) {
+        FileSKVIterator reader = entry.getKey();
+        String filename = entry.getValue();
         InterruptibleIterator iter;
 
         FileSKVIterator source = reader;
@@ -570,10 +564,11 @@ public class FileManager {
       for (FileDataSource fds : dataSources)
         files.add(fds.file);
 
-      List<FileSKVIterator> newlyReservedReaders = openFiles(files);
+      Map<FileSKVIterator,String> newlyReservedReaders = openFiles(files);
       Map<String,List<FileSKVIterator>> map = new HashMap<>();
-      for (FileSKVIterator reader : newlyReservedReaders) {
-        String fileName = getReservedReadeFilename(reader);
+      for (Entry<FileSKVIterator,String> entry : newlyReservedReaders.entrySet()) {
+        FileSKVIterator reader = entry.getKey();
+        String fileName = entry.getValue();
         List<FileSKVIterator> list = map.get(fileName);
         if (list == null) {
           list = new LinkedList<>();

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

Mime
View raw message