HBASE-3211 Key (Index) Only Fetches

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1033777 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Gray 2010-11-10 23:35:09 +00:00
parent 63c6827376
commit 13f7d93b56
5 changed files with 146 additions and 2 deletions

View File

@ -1205,6 +1205,7 @@ Release 0.90.0 - Unreleased
HBASE-3013 Tool to verify data in two clusters HBASE-3013 Tool to verify data in two clusters
HBASE-2896 Retain assignment information between cluster HBASE-2896 Retain assignment information between cluster
shutdown/startup shutdown/startup
HBASE-3211 Key (Index) Only Fetches
OPTIMIZATIONS OPTIMIZATIONS

View File

@ -1188,6 +1188,25 @@ public class KeyValue implements Writable, HeapSize {
return getQualifierLength() == 0; return getQualifierLength() == 0;
} }
/**
* 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. * Splits a column in family:qualifier form into separate byte arrays.
* <p> * <p>

View File

@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.DependentColumnFilter; import org.apache.hadoop.hbase.filter.DependentColumnFilter;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.filter.InclusiveStopFilter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.hadoop.hbase.filter.PageFilter; import org.apache.hadoop.hbase.filter.PageFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.hbase.filter.QualifierFilter;
@ -201,6 +202,8 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
addToMap(Increment.class, code++); addToMap(Increment.class, code++);
addToMap(KeyOnlyFilter.class, code++);
} }
private Class<?> declaredClass; private Class<?> declaredClass;

View File

@ -322,4 +322,34 @@ public class TestKeyValue extends TestCase {
assertKVLess(c, kvA_1, lastOnRowA); assertKVLess(c, kvA_1, lastOnRowA);
assertKVLess(c, firstOnRowA, 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);
}
} }

View File

@ -1255,8 +1255,6 @@ public class TestFilter extends HBaseTestCase {
" rows", expectedRows, i); " rows", expectedRows, i);
} }
private void verifyScanNoEarlyOut(Scan s, long expectedRows, private void verifyScanNoEarlyOut(Scan s, long expectedRows,
long expectedKeys) long expectedKeys)
throws IOException { throws IOException {
@ -1314,6 +1312,42 @@ public class TestFilter extends HBaseTestCase {
kvs.length, idx); 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 { public void testColumnPaginationFilter() throws Exception {
@ -1406,4 +1440,61 @@ public class TestFilter extends HBaseTestCase {
verifyScan(s, expectedRows, 0); verifyScan(s, expectedRows, 0);
this.verifyScanFull(s, expectedKVs4); 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);
}
} }