hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject svn commit: r1462231 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/io/ main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/io/encoding/
Date Thu, 28 Mar 2013 17:39:24 GMT
Author: tedyu
Date: Thu Mar 28 17:39:23 2013
New Revision: 1462231

URL: http://svn.apache.org/r1462231
Log:
HBASE-8198 Backport HBASE-8063(Filter HFiles based on first/last key) into 0.94 (Liang Xie)


Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java?rev=1462231&r1=1462230&r2=1462231&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
(original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
Thu Mar 28 17:39:23 2013
@@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@@ -283,6 +284,11 @@ public class HalfStoreFileReader extends
   }
 
   @Override
+  public boolean passesKeyRangeFilter(Scan scan) {
+    return true;
+  }
+  
+  @Override
   public byte[] getLastKey() {
     if (top) {
       return super.getLastKey();

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1462231&r1=1462230&r2=1462231&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
(original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
Thu Mar 28 17:39:23 2013
@@ -1610,6 +1610,28 @@ public class StoreFile extends SchemaCon
       return true;
     }
 
+    /**
+     * Checks whether the given scan rowkey range overlaps with the current storefile's
+     * @param scan the scan specification. Used to determine the rowkey range.
+     * @return true if there is overlap, false otherwise
+     */
+    public boolean passesKeyRangeFilter(Scan scan) {
+      if (this.getFirstKey() == null || this.getLastKey() == null) {
+        // the file is empty
+        return false;
+      }
+      if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
+          && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
+        return true;
+      }
+      KeyValue startKeyValue = KeyValue.createFirstOnRow(scan.getStartRow());
+      KeyValue stopKeyValue = KeyValue.createLastOnRow(scan.getStopRow());
+      boolean nonOverLapping = (getComparator().compare(this.getFirstKey(),
+        stopKeyValue.getKey()) > 0 && !Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW))
+          || getComparator().compare(this.getLastKey(), startKeyValue.getKey()) < 0;
+      return !nonOverLapping;
+    }
+
     public Map<byte[], byte[]> loadFileInfo() throws IOException {
       Map<byte [], byte []> fi = reader.loadFileInfo();
 

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java?rev=1462231&r1=1462230&r2=1462231&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
(original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
Thu Mar 28 17:39:23 2013
@@ -368,9 +368,8 @@ public class StoreFileScanner implements
   }
 
   @Override
-  public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
-      long oldestUnexpiredTS) {
-    return reader.passesTimerangeFilter(scan, oldestUnexpiredTS) &&
-        reader.passesBloomFilter(scan, columns);
+  public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS)
{
+    return reader.passesTimerangeFilter(scan, oldestUnexpiredTS)
+        && reader.passesKeyRangeFilter(scan) && reader.passesBloomFilter(scan,
columns);
   }
 }

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java?rev=1462231&r1=1462230&r2=1462231&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
(original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
Thu Mar 28 17:39:23 2013
@@ -238,10 +238,18 @@ public class TestChangingEncoding {
   private void compactAndWait() throws IOException, InterruptedException {
     LOG.debug("Compacting table " + tableName);
     admin.majorCompact(tableName);
-    Threads.sleepWithoutInterrupt(500);
     HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
+    
+    // Waiting for the compaction to start, at least .5s.
+    final long maxWaitime = System.currentTimeMillis() + 500;
+    boolean cont;
+    do {
+      cont = rs.compactSplitThread.getCompactionQueueSize() == 0;
+      Threads.sleep(1);
+    } while (cont && System.currentTimeMillis() < maxWaitime);
+    
     while (rs.compactSplitThread.getCompactionQueueSize() > 0) {
-      Threads.sleep(50);
+      Threads.sleep(5);
     }
     LOG.debug("Compaction queue size reached 0, continuing");
   }



Mime
View raw message