You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:03:35 UTC

svn commit: r1181377 - in /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase: KeyValue.java filter/TimestampsFilter.java

Author: nspiegelberg
Date: Tue Oct 11 02:03:35 2011
New Revision: 1181377

URL: http://svn.apache.org/viewvc?rev=1181377&view=rev
Log:
Reading only the required messages in a thread

Summary:
This patch enables reading only the specific messages (timestamps) in a thread
rather than scanning the entire action log from beginning to  end. Is especially
useful for sparse threads. If one message of the thread was recent and another
was old, we don't want to read everything in between.

Test Plan:
Existing timestamp filters go through this code path and all the tests are
passing.

More testing may be required.

DiffCamp Revision: 158667
Reviewed By: kannan
CC: kannan, pkhaitan, hbase@lists
Tasks:
#391726: Increase the speed for message reads by skipping between timestamps

Revert Plan:
OK

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1181377&r1=1181376&r2=1181377&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/KeyValue.java Tue Oct 11 02:03:35 2011
@@ -1606,6 +1606,32 @@ public class KeyValue implements Writabl
   }
 
   /**
+   * Create a KeyValue for the specified row, family, qualifier, timestamp that
+   * would be smaller than or equal to all other possible KeyValues that have
+   * the same row, family, qualifier, timestamp.
+   * Used for seeking.
+   * @param row row key
+   * @param roffset row offset
+   * @param rlength row length
+   * @param family family name
+   * @param foffset family offset
+   * @param flength family length
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @param timestamp timestamp
+   * @return First possible key on passed row, family, qualifier, timestamp
+   */
+  public static KeyValue createFirstOnRow(final byte [] row,
+      final int roffset, final int rlength, final byte [] family,
+      final int foffset, final int flength, final byte [] qualifier,
+      final int qoffset, final int qlength, long timestamp) {
+    return new KeyValue(row, roffset, rlength, family,
+        foffset, flength, qualifier, qoffset, qlength,
+        timestamp, Type.Maximum, null, 0, 0);
+  }
+
+  /**
    * @param b
    * @return A KeyValue made of a byte array that holds the key-only part.
    * Needed to convert hfile index members to KeyValues.

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java?rev=1181377&r1=1181376&r2=1181377&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java Tue Oct 11 02:03:35 2011
@@ -67,7 +67,7 @@ public class TimestampsFilter extends Fi
       // to be lesser than all of the other values.
       return ReturnCode.NEXT_COL;
     }
-    return ReturnCode.SKIP;
+    return ReturnCode.SEEK_NEXT_USING_HINT;
   }
 
   @Override
@@ -88,4 +88,14 @@ public class TimestampsFilter extends Fi
       out.writeLong(timestamp);
     }
   }
+
+  @Override
+  public KeyValue getNextKeyHint(KeyValue kv) {
+    Long nextTimestampObject = timestamps.lower(kv.getTimestamp());
+    long nextTimestamp = nextTimestampObject != null ? nextTimestampObject : 0;
+    return KeyValue.createFirstOnRow(kv.getBuffer(), kv.getRowOffset(), kv
+        .getRowLength(), kv.getBuffer(), kv.getFamilyOffset(), kv
+        .getFamilyLength(), kv.getBuffer(), kv.getQualifierOffset(), kv
+        .getQualifierLength(), nextTimestamp);
+  }
 }