HBASE-4394 Add support for seeking hints to FilterList
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1170422 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
bf5a6641bf
commit
a902f9ab1c
|
@ -260,6 +260,7 @@ Release 0.91.0 - Unreleased
|
|||
HTable client (Lars Hofhansl via garyh)
|
||||
HBASE-4340 Hbase can't balance if ServerShutdownHandler encountered
|
||||
exception (Jinchao Gao)
|
||||
HBASE-4394 Add support for seeking hints to FilterList
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
|
||||
|
|
|
@ -274,6 +274,31 @@ public class FilterList implements Filter {
|
|||
|
||||
@Override
|
||||
public KeyValue getNextKeyHint(KeyValue currentKV) {
|
||||
return null;
|
||||
KeyValue keyHint = null;
|
||||
for (Filter filter : filters) {
|
||||
KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
|
||||
if (curKeyHint == null && operator == Operator.MUST_PASS_ONE) {
|
||||
// If we ever don't have a hint and this is must-pass-one, then no hint
|
||||
return null;
|
||||
}
|
||||
if (curKeyHint != null) {
|
||||
// If this is the first hint we find, set it
|
||||
if (keyHint == null) {
|
||||
keyHint = curKeyHint;
|
||||
continue;
|
||||
}
|
||||
// There is an existing hint
|
||||
if (operator == Operator.MUST_PASS_ALL &&
|
||||
KeyValue.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
|
||||
// If all conditions must pass, we can keep the max hint
|
||||
keyHint = curKeyHint;
|
||||
} else if (operator == Operator.MUST_PASS_ONE &&
|
||||
KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
|
||||
// If any condition can pass, we need to keep the min hint
|
||||
keyHint = curKeyHint;
|
||||
}
|
||||
}
|
||||
}
|
||||
return keyHint;
|
||||
}
|
||||
}
|
|
@ -21,17 +21,21 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutput;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.filter.FilterList.Operator;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* Tests filter sets
|
||||
*
|
||||
|
@ -225,4 +229,100 @@ public class TestFilterList extends TestCase {
|
|||
|
||||
// TODO: Run TESTS!!!
|
||||
}
|
||||
|
||||
/**
|
||||
* Test pass-thru of hints.
|
||||
*/
|
||||
public void testHintPassThru() throws Exception {
|
||||
|
||||
final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
|
||||
final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
|
||||
null, null);
|
||||
|
||||
Filter filterNoHint = new FilterBase() {
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput arg0) throws IOException {}
|
||||
};
|
||||
|
||||
Filter filterMinHint = new FilterBase() {
|
||||
@Override
|
||||
public KeyValue getNextKeyHint(KeyValue currentKV) {
|
||||
return minKeyValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput arg0) throws IOException {}
|
||||
};
|
||||
|
||||
Filter filterMaxHint = new FilterBase() {
|
||||
@Override
|
||||
public KeyValue getNextKeyHint(KeyValue currentKV) {
|
||||
return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput arg0) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput arg0) throws IOException {}
|
||||
};
|
||||
|
||||
// MUST PASS ONE
|
||||
|
||||
// Should take the min if given two hints
|
||||
FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
|
||||
Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
minKeyValue));
|
||||
|
||||
// Should have no hint if any filter has no hint
|
||||
filterList = new FilterList(Operator.MUST_PASS_ONE,
|
||||
Arrays.asList(
|
||||
new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
|
||||
assertNull(filterList.getNextKeyHint(null));
|
||||
filterList = new FilterList(Operator.MUST_PASS_ONE,
|
||||
Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
|
||||
assertNull(filterList.getNextKeyHint(null));
|
||||
|
||||
// Should give max hint if its the only one
|
||||
filterList = new FilterList(Operator.MUST_PASS_ONE,
|
||||
Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
maxKeyValue));
|
||||
|
||||
// MUST PASS ALL
|
||||
|
||||
// Should take the max if given two hints
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
maxKeyValue));
|
||||
|
||||
// Should have max hint even if a filter has no hint
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(
|
||||
new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
maxKeyValue));
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
maxKeyValue));
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
minKeyValue));
|
||||
|
||||
// Should give min hint if its the only one
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
|
||||
minKeyValue));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue