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:09:04 UTC

svn commit: r1181445 - in /hbase/branches/0.89/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/filter/ main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/ test/java/org/apache/hadoop/hbase/filter/

Author: nspiegelberg
Date: Tue Oct 11 02:09:03 2011
New Revision: 1181445

URL: http://svn.apache.org/viewvc?rev=1181445&view=rev
Log:
HBASE-3232: KeyOnlyFilter Improvements

Summary:
1. add option to get value length from KeyOnlyFilter
2. fix existing bug in KeyOnlyFilter

Test Plan:
mvn clean test

DiffCamp Revision: 182206
Reviewed By: aravind
CC: aravind
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/KeyOnlyFilter.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.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=1181445&r1=1181444&r2=1181445&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:09:03 2011
@@ -1151,13 +1151,19 @@ public class KeyValue implements Writabl
    * <p>
    * This method is used by {@link KeyOnlyFilter} and is an advanced feature of
    * KeyValue, proceed with caution.
+   * @param lenAsVal replace value with the actual value length (false=empty)
    */
-  public void convertToKeyOnly() {
-    // KV format:  <keylen/4><valuelen/4><key/keylen><value/valuelen>
-    // Rebuild as: <keylen/4><0/4><key/keylen>
-    byte [] newBuffer = new byte[getKeyLength() + (2 * Bytes.SIZEOF_INT)];
-    System.arraycopy(this.bytes, this.offset, newBuffer, 0, newBuffer.length);
-    Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, 0);
+  public void convertToKeyOnly(boolean lenAsVal) {
+    // KV format:  <keylen:4><valuelen:4><key:keylen><value:valuelen>
+    // Rebuild as: <keylen:4><0:4><key:keylen>
+    int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0;
+    byte [] newBuffer = new byte[getKeyLength() + (2 * Bytes.SIZEOF_INT) + dataLen];
+    System.arraycopy(this.bytes, this.offset, newBuffer, 0,
+        Math.min(newBuffer.length,this.length));
+    Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
+    if (lenAsVal) {
+      Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength());
+    }
     this.bytes = newBuffer;
     this.offset = 0;
     this.length = newBuffer.length;

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1181445&r1=1181444&r2=1181445&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Tue Oct 11 02:09:03 2011
@@ -23,6 +23,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * A filter that will only return the key component of each KV (the value will
@@ -33,11 +34,13 @@ import org.apache.hadoop.hbase.KeyValue;
  */
 public class KeyOnlyFilter extends FilterBase {
 
-  public KeyOnlyFilter() {}
+  boolean lenAsVal;
+  public KeyOnlyFilter() { this(false); }
+  public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
 
   @Override
   public ReturnCode filterKeyValue(KeyValue kv) {
-    kv.convertToKeyOnly();
+    kv.convertToKeyOnly(this.lenAsVal);
     return ReturnCode.INCLUDE;
   }
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java?rev=1181445&r1=1181444&r2=1181445&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java Tue Oct 11 02:09:03 2011
@@ -237,12 +237,14 @@ class StoreScanner implements KeyValueSc
     KeyValue kv;
     List<KeyValue> results = new ArrayList<KeyValue>();
     LOOP: while((kv = this.heap.peek()) != null) {
-      ScanQueryMatcher.MatchCode qcode = matcher.match(kv);
+      // kv is no longer immutable due to KeyOnlyFilter! use copy for safety
+      KeyValue copyKv = new KeyValue(kv.getBuffer(), kv.getOffset(), kv.getLength());
+      ScanQueryMatcher.MatchCode qcode = matcher.match(copyKv);
       //DebugPrint.println("SS peek kv = " + kv + " with qcode = " + qcode);
       switch(qcode) {
         case INCLUDE:
-          KeyValue next = this.heap.next();
-          results.add(next);
+          results.add(copyKv);
+          this.heap.next();
           if (limit > 0 && (results.size() == limit)) {
             break LOOP;
           }

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java?rev=1181445&r1=1181444&r2=1181445&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java Tue Oct 11 02:09:03 2011
@@ -328,28 +328,20 @@ public class TestKeyValue extends TestCa
     byte [] value = Bytes.toBytes("a real value");
     byte [] evalue = new byte[0]; // empty value
 
-    // verify key with a non-empty value works
-    KeyValue kv1 = new KeyValue(rowA, family, qualA, ts, value);
-    KeyValue kv1ko = kv1.clone();
-    assertTrue(kv1.equals(kv1ko));
-    kv1ko.convertToKeyOnly();
-    // keys are still the same
-    assertTrue(kv1.equals(kv1ko));
-    // but values are not
-    assertTrue(kv1.getValue().length != 0);
-    assertTrue(kv1ko.getValue().length == 0);
-
-    // verify key with an already-empty value works
-    KeyValue kv2 = new KeyValue(rowA, family, qualA, ts, evalue);
-    KeyValue kv2ko = kv2.clone();
-    assertTrue(kv2.equals(kv2ko));
-    kv2ko.convertToKeyOnly();
-    // they should still be equal
-    assertTrue(kv2.equals(kv2ko));
-    // but they should have different underlying byte arrays
-    assertFalse(kv2.getBuffer() == kv2ko.getBuffer());
-    // both with 0 length values
-    assertTrue(kv2.getValue().length == 0);
-    assertTrue(kv2ko.getValue().length == 0);
+    for (byte[] val : new byte[][]{value, evalue}) {
+      for (boolean useLen : new boolean[]{false,true}) {
+        KeyValue kv1 = new KeyValue(rowA, family, qualA, ts, val);
+        KeyValue kv1ko = kv1.clone();
+        assertTrue(kv1.equals(kv1ko));
+        kv1ko.convertToKeyOnly(useLen);
+        // keys are still the same
+        assertTrue(kv1.equals(kv1ko));
+        // but values are not
+        assertTrue(kv1ko.getValue().length == (useLen?Bytes.SIZEOF_INT:0));
+        if (useLen) {
+          assertEquals(kv1.getValueLength(), Bytes.toInt(kv1ko.getValue()));
+        }
+      }
+    }
   }
 }

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1181445&r1=1181444&r2=1181445&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Tue Oct 11 02:09:03 2011
@@ -1102,14 +1102,14 @@ public class TestFilter extends HBaseTes
         kvs.length, idx);
   }
 
-  private void verifyScanFullNoValues(Scan s, KeyValue [] kvs)
+  private void verifyScanFullNoValues(Scan s, KeyValue [] kvs, boolean useLen)
   throws IOException {
     InternalScanner scanner = this.region.getScanner(s);
     List<KeyValue> results = new ArrayList<KeyValue>();
     int row = 0;
     int idx = 0;
-    for (boolean done = true; done; row++) {
-      done = scanner.next(results);
+    for (boolean more = true; more; row++) {
+      more = scanner.next(results);
       Arrays.sort(results.toArray(new KeyValue[results.size()]),
           KeyValue.COMPARATOR);
       if(results.isEmpty()) break;
@@ -1126,9 +1126,20 @@ public class TestFilter extends HBaseTes
             Bytes.equals(kv.getFamily(), kvs[idx].getFamily()));
         assertTrue("Qualifier mismatch",
             Bytes.equals(kv.getQualifier(), kvs[idx].getQualifier()));
-        assertFalse("Value match (expecting no value in result)",
+        assertFalse("Should not have returned whole value",
             Bytes.equals(kv.getValue(), kvs[idx].getValue()));
-        assertTrue("Value in result is not empty", kv.getValue().length == 0);
+        if (useLen) {
+          assertEquals("Value in result is not SIZEOF_INT",
+                     kv.getValue().length, Bytes.SIZEOF_INT);
+          LOG.info("idx = "  + idx + ", len=" + kvs[idx].getValueLength()
+              + ", actual=" +  Bytes.toInt(kv.getValue()));
+          assertEquals("Scan value should be the length of the actual value. ",
+                     kvs[idx].getValueLength(), Bytes.toInt(kv.getValue()) );
+          LOG.info("good");
+        } else {
+          assertEquals("Value in result is not empty",
+                     kv.getValue().length, 0);
+        }
         idx++;
       }
       results.clear();
@@ -1282,9 +1293,11 @@ public class TestFilter extends HBaseTes
     // Grab all 6 rows
     long expectedRows = 6;
     long expectedKeys = this.colsPerRow;
-    Scan s = new Scan();
-    s.setFilter(new KeyOnlyFilter());
-    verifyScan(s, expectedRows, expectedKeys);
-    verifyScanFullNoValues(s, expectedKVs);
+    for (boolean useLen : new boolean[]{false,true}) {
+      Scan s = new Scan();
+      s.setFilter(new KeyOnlyFilter(useLen));
+      verifyScan(s, expectedRows, expectedKeys);
+      verifyScanFullNoValues(s, expectedKVs, useLen);
+    }
   }
 }