You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/03/13 07:04:31 UTC

svn commit: r1455804 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/regionserver/

Author: stack
Date: Wed Mar 13 06:04:31 2013
New Revision: 1455804

URL: http://svn.apache.org/r1455804
Log:
HBASE-7952 Remove update() and Improve ExplicitColumnTracker performance

Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestExplicitColumnTracker.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java?rev=1455804&r1=1455803&r2=1455804&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java Wed Mar 13 06:04:31 2013
@@ -31,12 +31,18 @@ import org.apache.hadoop.hbase.regionser
  * Currently there are two different types of Store/Family-level queries.
  * <ul><li>{@link ExplicitColumnTracker} is used when the query specifies
  * one or more column qualifiers to return in the family.
+ * <ul><li>{@link ScanWildcardColumnTracker} is used when no columns are
+ * explicitly specified.
  * <p>
- * This class is utilized by {@link ScanQueryMatcher} through two methods:
+ * This class is utilized by {@link ScanQueryMatcher} mainly through two methods:
  * <ul><li>{@link #checkColumn} is called when a Put satisfies all other
- * conditions of the query.  This method returns a {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode} to define
- * what action should be taken.
- * <li>{@link #update} is called at the end of every StoreFile or memstore.
+ * conditions of the query.
+ * <ul><li>{@link #getNextRowOrNextColumn} is called whenever ScanQueryMatcher
+ * believes that the current column should be skipped (by timestamp, filter etc.)
+ * <p>
+ * These two methods returns a 
+ * {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode}
+ * to define what action should be taken.
  * <p>
  * This class is NOT thread-safe as queries are never multi-threaded
  */
@@ -61,11 +67,6 @@ public interface ColumnTracker {
       throws IOException;
 
   /**
-   * Updates internal variables in between files
-   */
-  public void update();
-
-  /**
    * Resets the Matcher
    */
   public void reset();

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java?rev=1455804&r1=1455803&r2=1455804&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java Wed Mar 13 06:04:31 2013
@@ -39,11 +39,15 @@ import org.apache.hadoop.hbase.util.Byte
  * between rows.
  *
  * <p>
- * This class is utilized by {@link ScanQueryMatcher} through two methods:
+ * This class is utilized by {@link ScanQueryMatcher} mainly through two methods:
  * <ul><li>{@link #checkColumn} is called when a Put satisfies all other
- * conditions of the query.  This method returns a {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode} to define
- * what action should be taken.
- * <li>{@link #update} is called at the end of every StoreFile or memstore.
+ * conditions of the query.
+ * <ul><li>{@link #getNextRowOrNextColumn} is called whenever ScanQueryMatcher
+ * believes that the current column should be skipped (by timestamp, filter etc.)
+ * <p>
+ * These two methods returns a 
+ * {@link org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode}
+ * to define what action should be taken.
  * <p>
  * This class is NOT thread-safe as queries are never multi-threaded
  */
@@ -59,7 +63,6 @@ public class ExplicitColumnTracker imple
   * column have been returned.
   */
   private final List<ColumnCount> columns;
-  private final List<ColumnCount> columnsToReuse;
   private int index;
   private ColumnCount column;
   /** Keeps track of the latest timestamp included for current column.
@@ -81,9 +84,8 @@ public class ExplicitColumnTracker imple
     this.minVersions = minVersions;
     this.oldestStamp = oldestUnexpiredTS;
     this.columns = new ArrayList<ColumnCount>(columns.size());
-    this.columnsToReuse = new ArrayList<ColumnCount>(columns.size());
     for(byte [] column : columns) {
-      this.columnsToReuse.add(new ColumnCount(column));
+      this.columns.add(new ColumnCount(column));
     }
     reset();
   }
@@ -92,7 +94,7 @@ public class ExplicitColumnTracker imple
    * Done when there are no more columns to match against.
    */
   public boolean done() {
-    return this.columns.size() == 0;
+    return this.index >= this.columns.size();
   }
 
   public ColumnCount getColumnHint() {
@@ -110,7 +112,7 @@ public class ExplicitColumnTracker imple
     assert !KeyValue.isDelete(type);
     do {
       // No more columns left, we are done with this query
-      if(this.columns.size() == 0) {
+      if(done()) {
         return ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW; // done_row
       }
 
@@ -136,14 +138,9 @@ public class ExplicitColumnTracker imple
         int count = this.column.increment();
         if(count >= maxVersions || (count >= minVersions && isExpired(timestamp))) {
           // Done with versions for this column
-          // Note: because we are done with this column, and are removing
-          // it from columns, we don't do a ++this.index. The index stays
-          // the same but the columns have shifted within the array such
-          // that index now points to the next column we are interested in.
-          this.columns.remove(this.index);
-
+          ++this.index;
           resetTS();
-          if (this.columns.size() == this.index) {
+          if (done()) {
             // We have served all the requested columns.
             this.column = null;
             return ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW;
@@ -172,7 +169,8 @@ public class ExplicitColumnTracker imple
       // of interest. Advance the ExplicitColumnTracker state to next
       // column of interest, and check again.
       if (ret <= -1) {
-        if (++this.index >= this.columns.size()) {
+        ++this.index;
+        if (done()) {
           // No more to match, do not include, done with this row.
           return ScanQueryMatcher.MatchCode.SEEK_NEXT_ROW; // done_row
         }
@@ -182,24 +180,13 @@ public class ExplicitColumnTracker imple
     } while(true);
   }
 
-  /**
-   * Called at the end of every StoreFile or memstore.
-   */
-  public void update() {
-    if(this.columns.size() != 0) {
-      this.index = 0;
-      this.column = this.columns.get(this.index);
-    } else {
-      this.index = -1;
-      this.column = null;
-    }
-  }
-
   // Called between every row.
   public void reset() {
-    buildColumnList();
     this.index = 0;
     this.column = this.columns.get(this.index);
+    for(ColumnCount col : this.columns) {
+      col.setCount(0);
+    }
     resetTS();
   }
 
@@ -219,14 +206,6 @@ public class ExplicitColumnTracker imple
     return timestamp < oldestStamp;
   }
 
-  private void buildColumnList() {
-    this.columns.clear();
-    this.columns.addAll(this.columnsToReuse);
-    for(ColumnCount col : this.columns) {
-      col.setCount(0);
-    }
-  }
-
   /**
    * This method is used to inform the column tracker that we are done with
    * this column. We may get this information from external filters or
@@ -241,24 +220,18 @@ public class ExplicitColumnTracker imple
       int compare = Bytes.compareTo(column.getBuffer(), column.getOffset(),
           column.getLength(), bytes, offset, length);
       resetTS();
-      if (compare == 0) {
-        this.columns.remove(this.index);
-        if (this.columns.size() == this.index) {
+      if (compare <= 0) {
+        ++this.index;
+        if (done()) {
           // Will not hit any more columns in this storefile
           this.column = null;
         } else {
           this.column = this.columns.get(this.index);
         }
-        return;
-      } else if ( compare <= -1) {
-        if(++this.index != this.columns.size()) {
-          this.column = this.columns.get(this.index);
-        } else {
-          this.column = null;
-        }
-      } else {
-        return;
+        if (compare <= -1)
+          continue;
       }
+      return;
     }
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java?rev=1455804&r1=1455803&r2=1455804&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java Wed Mar 13 06:04:31 2013
@@ -142,13 +142,6 @@ public class ScanWildcardColumnTracker i
   }
 
   @Override
-  public void update() {
-    // no-op, shouldn't even be called
-    throw new UnsupportedOperationException(
-        "ScanWildcardColumnTracker.update should never be called!");
-  }
-
-  @Override
   public void reset() {
     columnBuffer = null;
     resetTSAndType();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestExplicitColumnTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestExplicitColumnTracker.java?rev=1455804&r1=1455803&r2=1455804&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestExplicitColumnTracker.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestExplicitColumnTracker.java Wed Mar 13 06:04:31 2013
@@ -171,7 +171,7 @@ public class TestExplicitColumnTracker e
       explicit.checkColumn(col, 0, col.length, 1, KeyValue.Type.Put.getCode(),
           false);
     }
-    explicit.update();
+    explicit.reset();
 
     for (int i = 1; i < 100000; i+=2) {
       byte [] col = Bytes.toBytes("col"+i);