HBASE-3433 Remove the KV copy of every KV in Scan; introduced by HBASE-3232

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1201094 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2011-11-11 23:19:28 +00:00
parent a51931de2d
commit 571e8f8520
11 changed files with 70 additions and 26 deletions

View File

@ -75,8 +75,9 @@ Release 0.92.0 - Unreleased
(Akash Ashok) (Akash Ashok)
HBASE-4503 Purge deprecated HBaseClusterTestCase HBASE-4503 Purge deprecated HBaseClusterTestCase
HBASE-4374 Up default regions size from 256M to 1G HBASE-4374 Up default regions size from 256M to 1G
HBASE-4648 Bytes.toBigDecimal() doesn't use offset (Brian Keller via Lars H) HBASE-4648 Bytes.toBigDecimal() doesn't use offset (Bryan Keller via Lars H)
HBASE-4715 Remove stale broke .rb scripts from bin dir HBASE-4715 Remove stale broke .rb scripts from bin dir
HBASE-3433 Remove the KV copy of every KV in Scan; introduced by HBASE-3232 (Lars H)
BUG FIXES BUG FIXES
HBASE-3280 YouAreDeadException being swallowed in HRS getMaster HBASE-3280 YouAreDeadException being swallowed in HRS getMaster

View File

@ -1313,15 +1313,11 @@ public class KeyValue implements Writable, HeapSize {
} }
/** /**
* Converts this KeyValue to only contain the key portion (the value is * Creates a new KeyValue that only contains the key portion (the value is
* changed to be null). This method does a full copy of the backing byte * set to be null).
* array and does not modify the original byte array of this KeyValue.
* <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) * @param lenAsVal replace value with the actual value length (false=empty)
*/ */
public void convertToKeyOnly(boolean lenAsVal) { public KeyValue createKeyOnly(boolean lenAsVal) {
// KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen> // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
// Rebuild as: <keylen:4><0:4><key:keylen> // Rebuild as: <keylen:4><0:4><key:keylen>
int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0; int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0;
@ -1332,9 +1328,7 @@ public class KeyValue implements Writable, HeapSize {
if (lenAsVal) { if (lenAsVal) {
Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength()); Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength());
} }
this.bytes = newBuffer; return new KeyValue(newBuffer);
this.offset = 0;
this.length = newBuffer.length;
} }
/** /**

View File

@ -87,7 +87,22 @@ public interface Filter extends Writable {
* @return code as described below * @return code as described below
* @see Filter.ReturnCode * @see Filter.ReturnCode
*/ */
public ReturnCode filterKeyValue(KeyValue v); public ReturnCode filterKeyValue(final KeyValue v);
/**
* Give the filter a chance to transform the passed KeyValue.
* If the KeyValue is changed a new KeyValue object must be returned.
* @see org.apache.hadoop.hbase.KeyValue#shallowCopy()
*
* The transformed KeyValue is what is eventually returned to the
* client. Most filters will return the passed KeyValue unchanged.
* @see org.apache.hadoop.hbase.filter.KeyOnlyFilter#transform(KeyValue)
* for an example of a transformation.
*
* @param v the KeyValue in question
* @return the changed KeyValue
*/
public KeyValue transform(final KeyValue v);
/** /**
* Return codes for filterValue(). * Return codes for filterValue().
@ -147,5 +162,5 @@ public interface Filter extends Writable {
* @return KeyValue which must be next seeked. return null if the filter is * @return KeyValue which must be next seeked. return null if the filter is
* not sure which key to seek to next. * not sure which key to seek to next.
*/ */
public KeyValue getNextKeyHint(KeyValue currentKV); public KeyValue getNextKeyHint(final KeyValue currentKV);
} }

View File

@ -77,6 +77,16 @@ public abstract class FilterBase implements Filter {
return ReturnCode.INCLUDE; return ReturnCode.INCLUDE;
} }
/**
* By default no transformation takes place
*
* @inheritDoc
*/
@Override
public KeyValue transform(KeyValue v) {
return v;
}
/** /**
* Filters that never filter by modifying the returned List of KeyValues can * Filters that never filter by modifying the returned List of KeyValues can
* inherit this implementation that does nothing. * inherit this implementation that does nothing.
@ -128,5 +138,5 @@ public abstract class FilterBase implements Filter {
*/ */
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) { public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
throw new IllegalArgumentException("This method has not been implemented"); throw new IllegalArgumentException("This method has not been implemented");
} }
} }

View File

@ -180,6 +180,15 @@ public class FilterList implements Filter {
return operator == Operator.MUST_PASS_ONE; return operator == Operator.MUST_PASS_ONE;
} }
@Override
public KeyValue transform(KeyValue v) {
KeyValue current = v;
for (Filter filter : filters) {
current = filter.transform(current);
}
return current;
}
@Override @Override
public ReturnCode filterKeyValue(KeyValue v) { public ReturnCode filterKeyValue(KeyValue v) {
ReturnCode rc = operator == Operator.MUST_PASS_ONE? ReturnCode rc = operator == Operator.MUST_PASS_ONE?

View File

@ -43,9 +43,8 @@ public class KeyOnlyFilter extends FilterBase {
public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; } public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
@Override @Override
public ReturnCode filterKeyValue(KeyValue kv) { public KeyValue transform(KeyValue kv) {
kv.convertToKeyOnly(this.lenAsVal); return kv.createKeyOnly(this.lenAsVal);
return ReturnCode.INCLUDE;
} }
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) { public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {

View File

@ -76,6 +76,11 @@ public class SkipFilter extends FilterBase {
return c; return c;
} }
@Override
public KeyValue transform(KeyValue v) {
return filter.transform(v);
}
public boolean filterRow() { public boolean filterRow() {
return filterRow; return filterRow;
} }

View File

@ -75,6 +75,11 @@ public class WhileMatchFilter extends FilterBase {
return c; return c;
} }
@Override
public KeyValue transform(KeyValue v) {
return filter.transform(v);
}
public boolean filterRow() { public boolean filterRow() {
boolean filterRow = this.filter.filterRow(); boolean filterRow = this.filter.filterRow();
changeFAR(filterRow); changeFAR(filterRow);

View File

@ -364,6 +364,14 @@ public class ScanQueryMatcher {
return this.startKey; return this.startKey;
} }
/**
*
* @return the Filter
*/
Filter getFilter() {
return this.filter;
}
public KeyValue getNextKeyHint(KeyValue kv) { public KeyValue getNextKeyHint(KeyValue kv) {
if (filter == null) { if (filter == null) {
return null; return null;

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -288,22 +289,21 @@ class StoreScanner extends NonLazyKeyValueScanner
store != null ? store.getComparator() : null; store != null ? store.getComparator() : null;
LOOP: while((kv = this.heap.peek()) != null) { LOOP: while((kv = this.heap.peek()) != null) {
// kv is no longer immutable due to KeyOnlyFilter! use copy for safety
KeyValue copyKv = kv.shallowCopy();
// Check that the heap gives us KVs in an increasing order. // Check that the heap gives us KVs in an increasing order.
if (prevKV != null && comparator != null if (prevKV != null && comparator != null
&& comparator.compare(prevKV, kv) > 0) { && comparator.compare(prevKV, kv) > 0) {
throw new IOException("Key " + prevKV + " followed by a " + throw new IOException("Key " + prevKV + " followed by a " +
"smaller key " + kv + " in cf " + store); "smaller key " + kv + " in cf " + store);
} }
prevKV = copyKv; prevKV = kv;
ScanQueryMatcher.MatchCode qcode = matcher.match(copyKv); ScanQueryMatcher.MatchCode qcode = matcher.match(kv);
switch(qcode) { switch(qcode) {
case INCLUDE: case INCLUDE:
case INCLUDE_AND_SEEK_NEXT_ROW: case INCLUDE_AND_SEEK_NEXT_ROW:
case INCLUDE_AND_SEEK_NEXT_COL: case INCLUDE_AND_SEEK_NEXT_COL:
results.add(copyKv); Filter f = matcher.getFilter();
results.add(f == null ? kv : f.transform(kv));
if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) { if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
if (!matcher.moreRowsMayExistAfter(kv)) { if (!matcher.moreRowsMayExistAfter(kv)) {

View File

@ -353,7 +353,7 @@ public class TestKeyValue extends TestCase {
assertKVLess(c, firstOnRowA, lastOnRowA); assertKVLess(c, firstOnRowA, lastOnRowA);
} }
public void testConvertToKeyOnly() throws Exception { public void testCreateKeyOnly() throws Exception {
long ts = 1; long ts = 1;
byte [] value = Bytes.toBytes("a real value"); byte [] value = Bytes.toBytes("a real value");
byte [] evalue = new byte[0]; // empty value byte [] evalue = new byte[0]; // empty value
@ -361,9 +361,7 @@ public class TestKeyValue extends TestCase {
for (byte[] val : new byte[][]{value, evalue}) { for (byte[] val : new byte[][]{value, evalue}) {
for (boolean useLen : new boolean[]{false,true}) { for (boolean useLen : new boolean[]{false,true}) {
KeyValue kv1 = new KeyValue(rowA, family, qualA, ts, val); KeyValue kv1 = new KeyValue(rowA, family, qualA, ts, val);
KeyValue kv1ko = kv1.clone(); KeyValue kv1ko = kv1.createKeyOnly(useLen);
assertTrue(kv1.equals(kv1ko));
kv1ko.convertToKeyOnly(useLen);
// keys are still the same // keys are still the same
assertTrue(kv1.equals(kv1ko)); assertTrue(kv1.equals(kv1ko));
// but values are not // but values are not