hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From la...@apache.org
Subject [2/2] hbase git commit: HBASE-12411 Optionally enable p-reads and private readers for compactions.
Date Tue, 18 Nov 2014 20:42:35 GMT
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/93ac96df
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/93ac96df
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/93ac96df

Branch: refs/heads/0.98
Commit: 93ac96dffca5110395d615c227c004473adb5bc0
Parents: 4dbc2f6
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:38:55 2014 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/93ac96df/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 0628d48..bea75b4 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
@@ -219,6 +219,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/93ac96df/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 d19fe15..e8d5f1d 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.DoNotRetryIOException;
@@ -123,12 +124,20 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     oldestUnexpiredTS = EnvironmentEdgeManager.currentTimeMillis() - ttl;
     this.minVersions = minVersions;
 
+    if (store != null && ((HStore)store).getHRegion() != null
+        && ((HStore)store).getHRegion().getBaseConf() != null) {
+      Configuration conf = ((HStore) store).getHRegion().getBaseConf();
+      this.scanUsePread = conf.getBoolean("hbase.storescanner.use.pread", scan.isSmall());
+    } else {
+      this.scanUsePread = scan.isSmall();
+    }
+
     // We look up row-column Bloom filters for multi-column queries as part of
     // the seek operation. However, we also look the row-column Bloom filter
     // 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/93ac96df/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 e04a715..e0a4ea1 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>();
@@ -93,13 +108,19 @@ public class DefaultCompactor extends Compactor {
       // Throw the exception;
       throw ioe;
     } finally {
-      if (writer != null) {
-        if (e != null) {
-          writer.close();
-        } else {
-          writer.appendMetadata(fd.maxSeqId, request.isMajor());
-          writer.close();
-          newFiles.add(writer.getPath());
+      try {
+        if (writer != null) {
+          if (e != null) {
+            writer.close();
+          } else {
+            writer.appendMetadata(fd.maxSeqId, request.isMajor());
+            writer.close();
+            newFiles.add(writer.getPath());
+          }
+        }
+      } finally {
+        for (StoreFile f : readersToClose) {
+          f.closeReader(true);
         }
       }
     }


Mime
View raw message