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:08:50 UTC

svn commit: r1181442 - 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/io/ test/java/org/apache/hadoop/hbase/ test/java/org/apache/hadoop/hbase/filter/

Author: nspiegelberg
Date: Tue Oct 11 02:08:50 2011
New Revision: 1181442

URL: http://svn.apache.org/viewvc?rev=1181442&view=rev
Log:
HBase KeyOnlyFilter to only return the Keys/Index not the values

Summary:
Adds a new filter, KeyOnlyFilter.  The idea is that this will make it so only
the key portion of all the KVs are returned.  Could imagine a few use cases
where you just need the keys/index not the values.  We have one where we have
giant rows with big values and want to just get the qualifiers/versions w/o
values.

Adds a new method in KeyValue, convertToKeyOnly().  From javadoc:

  /**
   * Converts this KeyValue to only contain the key portion (the value is
   * changed to be null).  This method does a full copy of the backing byte
   * array and does not modify the original byte array of this KeyValue.
   * <p>
   * This method is used by {@link KeyOnlyFilter} and is an advanced feature of
   * KeyValue, proceed with caution.
   */

Test Plan:
Test of the KV method added to TestKeyValue.  Test of KeyOnlyFilter added to
TestFilter.  Both passing.

DiffCamp Revision: 181022
Reviewed By: kannan
Commenters: nspiegelberg
CC: nspiegelberg, pmalik, achao, kannan, hbase@lists
Revert Plan:
OK

Added:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
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/io/HbaseObjectWritable.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=1181442&r1=1181441&r2=1181442&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:08:50 2011
@@ -1145,6 +1145,25 @@ public class KeyValue implements Writabl
   }
 
   /**
+   * Converts this KeyValue to only contain the key portion (the value is
+   * changed to be null).  This method does a full copy of the backing byte
+   * array and does not modify the original byte array of this KeyValue.
+   * <p>
+   * This method is used by {@link KeyOnlyFilter} and is an advanced feature of
+   * KeyValue, proceed with caution.
+   */
+  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);
+    this.bytes = newBuffer;
+    this.offset = 0;
+    this.length = newBuffer.length;
+  }
+
+  /**
    * Splits a column in family:qualifier form into separate byte arrays.
    * <p>
    * Not recommend to be used as this is old-style API.

Added: 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=1181442&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Tue Oct 11 02:08:50 2011
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * A filter that will only return the key component of each KV (the value will
+ * be rewritten as empty).
+ * <p>
+ * This filter can be used to grab all of the keys without having to also grab
+ * the values.
+ */
+public class KeyOnlyFilter extends FilterBase {
+
+  public KeyOnlyFilter() {}
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue kv) {
+    kv.convertToKeyOnly();
+    return ReturnCode.INCLUDE;
+  }
+
+  public void write(DataOutput out) throws IOException {}
+
+  public void readFields(DataInput in) throws IOException {}
+}

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java?rev=1181442&r1=1181441&r2=1181442&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java Tue Oct 11 02:08:50 2011
@@ -172,6 +172,9 @@ public class HbaseObjectWritable impleme
     // List
     addToMap(List.class, code++);
     addToMap(ColumnPrefixFilter.class, code++);
+
+    addToMap(KeyOnlyFilter.class, code++);
+
   }
 
   private Class<?> declaredClass;

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=1181442&r1=1181441&r2=1181442&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:08:50 2011
@@ -322,4 +322,34 @@ public class TestKeyValue extends TestCa
     assertKVLess(c, kvA_1, lastOnRowA);
     assertKVLess(c, firstOnRowA, lastOnRowA);
   }
+
+  public void testConvertToKeyOnly() throws Exception {
+    long ts = 1;
+    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);
+  }
 }

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=1181442&r1=1181441&r2=1181442&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:08:50 2011
@@ -1045,8 +1045,6 @@ public class TestFilter extends HBaseTes
         " rows", expectedRows, i);
   }
 
-
-
   private void verifyScanNoEarlyOut(Scan s, long expectedRows,
       long expectedKeys)
   throws IOException {
@@ -1104,6 +1102,42 @@ public class TestFilter extends HBaseTes
         kvs.length, idx);
   }
 
+  private void verifyScanFullNoValues(Scan s, KeyValue [] kvs)
+  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);
+      Arrays.sort(results.toArray(new KeyValue[results.size()]),
+          KeyValue.COMPARATOR);
+      if(results.isEmpty()) break;
+      assertTrue("Scanned too many keys! Only expected " + kvs.length +
+          " total but already scanned " + (results.size() + idx) +
+          (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
+          kvs.length >= idx + results.size());
+      for(KeyValue kv : results) {
+        LOG.info("row=" + row + ", result=" + kv.toString() +
+            ", match=" + kvs[idx].toString());
+        assertTrue("Row mismatch",
+            Bytes.equals(kv.getRow(), kvs[idx].getRow()));
+        assertTrue("Family mismatch",
+            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)",
+            Bytes.equals(kv.getValue(), kvs[idx].getValue()));
+        assertTrue("Value in result is not empty", kv.getValue().length == 0);
+        idx++;
+      }
+      results.clear();
+    }
+    LOG.info("Looked at " + row + " rows with " + idx + " keys");
+    assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
+        kvs.length, idx);
+  }
+
 
   public void testColumnPaginationFilter() throws Exception {
 
@@ -1196,4 +1230,61 @@ public class TestFilter extends HBaseTes
       verifyScan(s, expectedRows, 0);
       this.verifyScanFull(s, expectedKVs4);
     }
+
+  public void testKeyOnlyFilter() throws Exception {
+
+    // KVs in first 6 rows
+    KeyValue [] expectedKVs = {
+      // testRowOne-0
+      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
+      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
+      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
+      // testRowOne-2
+      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
+      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
+      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
+      // testRowOne-3
+      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
+      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
+      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
+      // testRowTwo-0
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
+      // testRowTwo-2
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
+      // testRowTwo-3
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
+    };
+
+    // 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);
+  }
 }