HBASE-3232 : Fix KeyOnlyFilter + Add Value Length (Nicolas via Ryan)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1035128 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ryan Rawson 2010-11-15 03:10:31 +00:00
parent ed06c534a0
commit 069fecaeb7
7 changed files with 100 additions and 45 deletions

View File

@ -679,6 +679,7 @@ Release 0.90.0 - Unreleased
HBASE-3221 Race between splitting and disabling
HBASE-3224 NPE in KeyValue$KVComparator.compare when compacting
HBASE-3233 Fix Long Running Stats
HBASE-3232 : Fix KeyOnlyFilter + Add Value Length (Nicolas via Ryan)
IMPROVEMENTS

View File

@ -1195,13 +1195,19 @@ public class KeyValue implements Writable, HeapSize {
* <p>
* This method is used by <code>KeyOnlyFilter</code> 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;

View File

@ -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,15 +34,21 @@ 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;
}
public void write(DataOutput out) throws IOException {}
public void readFields(DataInput in) throws IOException {}
public void write(DataOutput out) throws IOException {
out.writeBoolean(this.lenAsVal);
}
public void readFields(DataInput in) throws IOException {
this.lenAsVal = in.readBoolean();
}
}

View File

@ -252,12 +252,14 @@ class StoreScanner implements KeyValueScanner, InternalScanner, ChangedReadersOb
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;
}

View File

@ -328,28 +328,20 @@ public class TestKeyValue extends TestCase {
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);
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();
kv1ko.convertToKeyOnly(useLen);
// 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);
assertTrue(kv1ko.getValue().length == (useLen?Bytes.SIZEOF_INT:0));
if (useLen) {
assertEquals(kv1.getValueLength(), Bytes.toInt(kv1ko.getValue()));
}
}
}
}
}

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.hbase.filter.RegexStringComparator;
@ -479,6 +480,39 @@ public class TestFromClientSide {
scanner.close();
}
@Test
public void testKeyOnlyFilter() throws Exception {
byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
byte [][] ROWS = makeN(ROW, 10);
byte [][] QUALIFIERS = {
Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
Bytes.toBytes("col2-<d2v1>-<d3v2>"), Bytes.toBytes("col3-<d2v1>-<d3v2>"),
Bytes.toBytes("col4-<d2v1>-<d3v2>"), Bytes.toBytes("col5-<d2v1>-<d3v2>"),
Bytes.toBytes("col6-<d2v1>-<d3v2>"), Bytes.toBytes("col7-<d2v1>-<d3v2>"),
Bytes.toBytes("col8-<d2v1>-<d3v2>"), Bytes.toBytes("col9-<d2v1>-<d3v2>")
};
for(int i=0;i<10;i++) {
Put put = new Put(ROWS[i]);
put.add(FAMILY, QUALIFIERS[i], VALUE);
ht.put(put);
}
Scan scan = new Scan();
scan.addFamily(FAMILY);
Filter filter = new KeyOnlyFilter(true);
scan.setFilter(filter);
ResultScanner scanner = ht.getScanner(scan);
int count = 0;
for(Result result : ht.getScanner(scan)) {
assertEquals(result.size(), 1);
assertEquals(result.raw()[0].getValueLength(), Bytes.SIZEOF_INT);
assertEquals(Bytes.toInt(result.raw()[0].getValue()), VALUE.length);
count++;
}
assertEquals(count, 10);
scanner.close();
}
/**
* Test simple table and non-existent row cases.
*/

View File

@ -1312,14 +1312,14 @@ public class TestFilter extends HBaseTestCase {
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;
@ -1336,9 +1336,20 @@ public class TestFilter extends HBaseTestCase {
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();
@ -1492,9 +1503,11 @@ public class TestFilter extends HBaseTestCase {
// Grab all 6 rows
long expectedRows = 6;
long expectedKeys = this.colsPerRow;
for (boolean useLen : new boolean[]{false,true}) {
Scan s = new Scan();
s.setFilter(new KeyOnlyFilter());
s.setFilter(new KeyOnlyFilter(useLen));
verifyScan(s, expectedRows, expectedKeys);
verifyScanFullNoValues(s, expectedKVs);
verifyScanFullNoValues(s, expectedKVs, useLen);
}
}
}