hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From la...@apache.org
Subject hbase git commit: HBASE-12411 Optionally enable p-reads and private readers for compactions.
Date Tue, 18 Nov 2014 20:43:04 GMT
Repository: hbase
Updated Branches:
  refs/heads/master 36c9170c5 -> 822bcce91


HBASE-12411 Optionally enable p-reads and private readers for compactions.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/822bcce9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/822bcce9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/822bcce9

Branch: refs/heads/master
Commit: 822bcce91e5d80f778c009dcc6d50089276859df
Parents: 36c9170
Author: Lars Hofhansl <larsh@apache.org>
Authored: Tue Nov 18 12:29:06 2014 -0800
Committer: Lars Hofhansl <larsh@apache.org>
Committed: Tue Nov 18 12:42:49 2014 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/StoreFile.java    | 12 +++++++
 .../hadoop/hbase/regionserver/StoreScanner.java | 10 ++++--
 .../compactions/DefaultCompactor.java           | 37 +++++++++++++++-----
 3 files changed, 48 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/822bcce9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 3e139a5..1626cc3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -220,6 +220,18 @@ public class StoreFile {
   }
 
   /**
+   * Clone
+   * @param other The StoreFile to clone from
+   */
+  public StoreFile(final StoreFile other) {
+    this.fs = other.fs;
+    this.fileInfo = other.fileInfo;
+    this.cacheConf = other.cacheConf;
+    this.cfBloomType = other.cfBloomType;
+    this.modificationTimeStamp = other.modificationTimeStamp;
+  }
+
+  /**
    * @return the StoreFile object associated to this StoreFile.
    *         null if the StoreFile is not a reference.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/822bcce9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 6d0098b..ab7370c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -127,10 +128,13 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
     if (store != null && ((HStore)store).getHRegion() != null
         && ((HStore)store).getHRegion().getBaseConf() != null) {
-      this.maxRowSize = ((HStore) store).getHRegion().getBaseConf().getLong(
-        HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT);
+      Configuration conf = ((HStore) store).getHRegion().getBaseConf();
+      this.maxRowSize =
+          conf.getLong(HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT);
+      this.scanUsePread = conf.getBoolean("hbase.storescanner.use.pread", scan.isSmall());
     } else {
       this.maxRowSize = HConstants.TABLE_MAX_ROWSIZE_DEFAULT;
+      this.scanUsePread = scan.isSmall();
     }
 
     // We look up row-column Bloom filters for multi-column queries as part of
@@ -138,7 +142,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // for multi-row (non-"get") scans because this is not done in
     // StoreFile.passesBloomFilter(Scan, SortedSet<byte[]>).
     useRowColBloom = numCol > 1 || (!isGet && numCol == 1);
-    this.scanUsePread = scan.isSmall();
+
     // The parallel-seeking is on :
     // 1) the config value is *true*
     // 2) store has more than one store file

http://git-wip-us.apache.org/repos/asf/hbase/blob/822bcce9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
index aae3968..cc03e09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -50,7 +51,21 @@ public class DefaultCompactor extends Compactor {
 
     // Find the smallest read point across all the Scanners.
     long smallestReadPoint = getSmallestReadPoint();
-    List<StoreFileScanner> scanners = createFileScanners(request.getFiles(), smallestReadPoint);
+
+    List<StoreFileScanner> scanners;
+    Collection<StoreFile> readersToClose;
+    if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", false)) {
+      // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
+      // HFileFiles, and their readers
+      readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
+      for (StoreFile f : request.getFiles()) {
+        readersToClose.add(new StoreFile(f));
+      }
+      scanners = createFileScanners(readersToClose, smallestReadPoint);
+    } else {
+      readersToClose = Collections.emptyList();
+      scanners = createFileScanners(request.getFiles(), smallestReadPoint);
+    }
 
     StoreFile.Writer writer = null;
     List<Path> newFiles = new ArrayList<Path>();
@@ -99,13 +114,19 @@ public class DefaultCompactor extends Compactor {
       throw ioe;
     }
     finally {
-      if (writer != null) {
-        if (e != null) {
-          writer.close();
-        } else {
-          writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
-          writer.close();
-          newFiles.add(writer.getPath());
+      try {
+        if (writer != null) {
+          if (e != null) {
+            writer.close();
+          } else {
+            writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
+            writer.close();
+            newFiles.add(writer.getPath());
+          }
+        }
+      } finally {
+        for (StoreFile f : readersToClose) {
+          f.closeReader(true);
         }
       }
     }


Mime
View raw message