keith-turner closed pull request #380: ACCUMULO-4805 Obtain filemanager lock 
once when opening files
URL: https://github.com/apache/accumulo/pull/380
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 a707251463..2aaad39ccd 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 FileManager(AccumuloServerContext context, 
VolumeManager fs, int maxOpen,
     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 @@ private static int 
countReaders(Map<String,List<OpenReader>> files) {
     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 @@ private void closeReaders(List<FileSKVIterator> 
filesToClose) {
     }
   }
 
-  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 @@ private void closeReaders(List<FileSKVIterator> 
filesToClose) {
       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 @@ private void closeReaders(List<FileSKVIterator> 
filesToClose) {
     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 @@ private synchronized String 
getReservedReadeFilename(FileSKVIterator reader) {
     // a synch block
     synchronized (this) {
 
-      filesToOpen = takeOpenFiles(files, reservedFiles, readersReserved);
+      filesToOpen = takeOpenFiles(files, readersReserved);
 
       int numOpen = countReaders(openFiles);
 
@@ -318,7 +312,6 @@ private synchronized String 
getReservedReadeFilename(FileSKVIterator reader) {
         // 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 @@ private synchronized String 
getReservedReadeFilename(FileSKVIterator reader) {
           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 @@ private synchronized String 
getReservedReadeFilename(FileSKVIterator reader) {
       reservedReaders.putAll(readersReserved);
     }
 
-    return reservedFiles;
+    return readersReserved;
   }
 
   private void releaseReaders(KeyExtent tablet, List<FileSKVIterator> readers, 
boolean sawIOException) {
@@ -481,14 +474,14 @@ public void setInterruptFlag(AtomicBoolean flag) {
       }
     }
 
-    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 void setInterruptFlag(AtomicBoolean flag) {
             + " 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 void setInterruptFlag(AtomicBoolean flag) {
         }
       }
 
-      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 synchronized void 
reattach(SamplerConfigurationImpl samplerConfig) throws
       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<>();


 

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to