HBASE-12296 Filters should work with ByteBufferedCell.
This commit is contained in:
parent
6c6c7c51f6
commit
ebdac4b52e
|
@ -19,10 +19,13 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -48,6 +51,11 @@ public class BinaryComparator extends ByteArrayComparable {
|
|||
return Bytes.compareTo(this.value, 0, this.value.length, value, offset, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ByteBuffer value, int offset, int length) {
|
||||
return -(ByteBufferUtils.compareTo(value, offset, length, this.value, 0, this.value.length));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The comparator serialized using pb
|
||||
*/
|
||||
|
|
|
@ -19,10 +19,13 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -50,6 +53,14 @@ public class BinaryPrefixComparator extends ByteArrayComparable {
|
|||
this.value.length <= length ? this.value.length : length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ByteBuffer value, int offset, int length) {
|
||||
if (this.value.length <= length) {
|
||||
length = this.value.length;
|
||||
}
|
||||
return -(ByteBufferUtils.compareTo(value, offset, length, this.value, 0, this.value.length));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The comparator serialized using pb
|
||||
*/
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -131,5 +133,28 @@ public class BitComparator extends ByteArrayComparable {
|
|||
}
|
||||
return b == 0 ? 1 : 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ByteBuffer value, int offset, int length) {
|
||||
if (length != this.value.length) {
|
||||
return 1;
|
||||
}
|
||||
int b = 0;
|
||||
//Iterating backwards is faster because we can quit after one non-zero byte.
|
||||
for (int i = length - 1; i >= 0 && b == 0; i--) {
|
||||
switch (bitOperator) {
|
||||
case AND:
|
||||
b = (this.value[i] & value.get(i + offset)) & 0xff;
|
||||
break;
|
||||
case OR:
|
||||
b = (this.value[i] | value.get(i + offset)) & 0xff;
|
||||
break;
|
||||
case XOR:
|
||||
b = (this.value[i] ^ value.get(i + offset)) & 0xff;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return b == 0 ? 1 : 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,8 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -118,19 +119,10 @@ public class ColumnPaginationFilter extends FilterBase {
|
|||
if (count >= limit) {
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
byte[] buffer = v.getQualifierArray();
|
||||
if (buffer == null) {
|
||||
return ReturnCode.SEEK_NEXT_USING_HINT;
|
||||
}
|
||||
int cmp = 0;
|
||||
// Only compare if no KV's have been seen so far.
|
||||
if (count == 0) {
|
||||
cmp = Bytes.compareTo(buffer,
|
||||
v.getQualifierOffset(),
|
||||
v.getQualifierLength(),
|
||||
this.columnOffset,
|
||||
0,
|
||||
this.columnOffset.length);
|
||||
cmp = CellComparator.compareQualifiers(v, this.columnOffset, 0, this.columnOffset.length);
|
||||
}
|
||||
if (cmp < 0) {
|
||||
return ReturnCode.SEEK_NEXT_USING_HINT;
|
||||
|
@ -152,9 +144,7 @@ public class ColumnPaginationFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return KeyValueUtil.createFirstOnRow(
|
||||
cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getFamilyArray(),
|
||||
cell.getFamilyOffset(), cell.getFamilyLength(), columnOffset, 0, columnOffset.length);
|
||||
return CellUtil.createFirstOnRowCol(cell, columnOffset, 0, columnOffset.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,12 +22,14 @@ package org.apache.hadoop.hbase.filter;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -59,26 +61,25 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell kv) {
|
||||
if (this.prefix == null || kv.getQualifierArray() == null) {
|
||||
public ReturnCode filterKeyValue(Cell cell) {
|
||||
if (this.prefix == null) {
|
||||
return ReturnCode.INCLUDE;
|
||||
} else {
|
||||
return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
return filterColumn(cell);
|
||||
}
|
||||
}
|
||||
|
||||
public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) {
|
||||
public ReturnCode filterColumn(Cell cell) {
|
||||
int qualifierLength = cell.getQualifierLength();
|
||||
if (qualifierLength < prefix.length) {
|
||||
int cmp = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, this.prefix, 0,
|
||||
qualifierLength);
|
||||
int cmp = compareQualifierPart(cell, qualifierLength, this.prefix);
|
||||
if (cmp <= 0) {
|
||||
return ReturnCode.SEEK_NEXT_USING_HINT;
|
||||
} else {
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
} else {
|
||||
int cmp = Bytes.compareTo(buffer, qualifierOffset, this.prefix.length, this.prefix, 0,
|
||||
this.prefix.length);
|
||||
int cmp = compareQualifierPart(cell, this.prefix.length, this.prefix);
|
||||
if (cmp < 0) {
|
||||
return ReturnCode.SEEK_NEXT_USING_HINT;
|
||||
} else if (cmp > 0) {
|
||||
|
@ -89,6 +90,15 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
private static int compareQualifierPart(Cell cell, int length, byte[] prefix) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPositionInByteBuffer(), length, prefix, 0, length);
|
||||
}
|
||||
return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0,
|
||||
length);
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 1,
|
||||
"Expected 1 but got: %s", filterArguments.size());
|
||||
|
@ -138,9 +148,7 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return KeyValueUtil.createFirstOnRow(
|
||||
cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getFamilyArray(),
|
||||
cell.getFamilyOffset(), cell.getFamilyLength(), prefix, 0, prefix.length);
|
||||
return CellUtil.createFirstOnRowCol(cell, prefix, 0, prefix.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,7 +25,8 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -124,15 +125,10 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell kv) {
|
||||
// TODO have a column compare method in Cell
|
||||
byte[] buffer = kv.getQualifierArray();
|
||||
int qualifierOffset = kv.getQualifierOffset();
|
||||
int qualifierLength = kv.getQualifierLength();
|
||||
int cmpMin = 1;
|
||||
|
||||
if (this.minColumn != null) {
|
||||
cmpMin = Bytes.compareTo(buffer, qualifierOffset, qualifierLength,
|
||||
this.minColumn, 0, this.minColumn.length);
|
||||
cmpMin = CellComparator.compareQualifiers(kv, this.minColumn, 0, this.minColumn.length);
|
||||
}
|
||||
|
||||
if (cmpMin < 0) {
|
||||
|
@ -147,8 +143,7 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
int cmpMax = Bytes.compareTo(buffer, qualifierOffset, qualifierLength,
|
||||
this.maxColumn, 0, this.maxColumn.length);
|
||||
int cmpMax = CellComparator.compareQualifiers(kv, this.maxColumn, 0, this.maxColumn.length);
|
||||
|
||||
if (this.maxColumnInclusive && cmpMax <= 0 ||
|
||||
!this.maxColumnInclusive && cmpMax < 0) {
|
||||
|
@ -224,10 +219,7 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), cell
|
||||
.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell
|
||||
.getFamilyLength(), this.minColumn, 0, len(this.minColumn));
|
||||
|
||||
return CellUtil.createFirstOnRowCol(cell, this.minColumn, 0, len(this.minColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -105,29 +106,58 @@ public abstract class CompareFilter extends FilterBase {
|
|||
return false;
|
||||
}
|
||||
|
||||
protected boolean doCompare(final CompareOp compareOp,
|
||||
final ByteArrayComparable comparator, final byte [] data,
|
||||
final int offset, final int length) {
|
||||
protected boolean compareRow(final CompareOp compareOp, final ByteArrayComparable comparator,
|
||||
final Cell cell) {
|
||||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = comparator.compareTo(data, offset, length);
|
||||
int compareResult = CellComparator.compareRow(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
protected boolean compareFamily(final CompareOp compareOp, final ByteArrayComparable comparator,
|
||||
final Cell cell) {
|
||||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellComparator.compareFamily(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
protected boolean compareQualifier(final CompareOp compareOp,
|
||||
final ByteArrayComparable comparator, final Cell cell) {
|
||||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellComparator.compareQualifier(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
protected boolean compareValue(final CompareOp compareOp, final ByteArrayComparable comparator,
|
||||
final Cell cell) {
|
||||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellComparator.compareValue(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
private boolean compare(final CompareOp compareOp, int compareResult) {
|
||||
switch (compareOp) {
|
||||
case LESS:
|
||||
return compareResult <= 0;
|
||||
case LESS_OR_EQUAL:
|
||||
return compareResult < 0;
|
||||
case EQUAL:
|
||||
return compareResult != 0;
|
||||
case NOT_EQUAL:
|
||||
return compareResult == 0;
|
||||
case GREATER_OR_EQUAL:
|
||||
return compareResult > 0;
|
||||
case GREATER:
|
||||
return compareResult >= 0;
|
||||
default:
|
||||
throw new RuntimeException("Unknown Compare op " +
|
||||
compareOp.name());
|
||||
case LESS:
|
||||
return compareResult <= 0;
|
||||
case LESS_OR_EQUAL:
|
||||
return compareResult < 0;
|
||||
case EQUAL:
|
||||
return compareResult != 0;
|
||||
case NOT_EQUAL:
|
||||
return compareResult == 0;
|
||||
case GREATER_OR_EQUAL:
|
||||
return compareResult > 0;
|
||||
case GREATER:
|
||||
return compareResult >= 0;
|
||||
default:
|
||||
throw new RuntimeException("Unknown Compare op " + compareOp.name());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -142,8 +142,7 @@ public class DependentColumnFilter extends CompareFilter {
|
|||
}
|
||||
// If it doesn't pass the op, skip it
|
||||
if (comparator != null
|
||||
&& doCompare(compareOp, comparator, c.getValueArray(), c.getValueOffset(),
|
||||
c.getValueLength()))
|
||||
&& compareValue(compareOp, comparator, c))
|
||||
return ReturnCode.SKIP;
|
||||
|
||||
stampSet.add(c.getTimestamp());
|
||||
|
|
|
@ -64,8 +64,7 @@ public class FamilyFilter extends CompareFilter {
|
|||
public ReturnCode filterKeyValue(Cell v) {
|
||||
int familyLength = v.getFamilyLength();
|
||||
if (familyLength > 0) {
|
||||
if (doCompare(this.compareOp, this.comparator, v.getFamilyArray(),
|
||||
v.getFamilyOffset(), familyLength)) {
|
||||
if (compareFamily(this.compareOp, this.comparator, v)) {
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,8 +63,6 @@ public abstract class FilterBase extends Filter {
|
|||
|
||||
@Override
|
||||
public boolean filterRowKey(Cell cell) throws IOException {
|
||||
// TODO when cell is backed by DirectByteBuffer, we would need to copy row bytes to temp byte[]
|
||||
// and call old method for BC.
|
||||
return filterRowKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -169,10 +170,9 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
}
|
||||
byte[] nextRowKey = tracker.nextRow();
|
||||
// We need to compare nextRowKey with currentCell
|
||||
int compareResult =
|
||||
Bytes.compareTo(nextRowKey, 0, nextRowKey.length, currentCell.getRowArray(),
|
||||
currentCell.getRowOffset(), currentCell.getRowLength());
|
||||
if ((reversed && compareResult > 0) || (!reversed && compareResult < 0)) {
|
||||
int compareResult = CellComparator.COMPARATOR.compareRows(currentCell, nextRowKey, 0,
|
||||
nextRowKey.length);
|
||||
if ((reversed && compareResult < 0) || (!reversed && compareResult > 0)) {
|
||||
// This can happen when we have multilpe filters and some other filter
|
||||
// returns next row with hint which is larger (smaller for reverse)
|
||||
// than the current (really?)
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -60,10 +61,8 @@ public class InclusiveStopFilter extends FilterBase {
|
|||
|
||||
public boolean filterRowKey(Cell firstRowCell) {
|
||||
// if stopRowKey is <= buffer, then true, filter row.
|
||||
int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
|
||||
firstRowCell.getRowArray(), firstRowCell.getRowOffset(), firstRowCell.getRowLength());
|
||||
|
||||
if(cmp < 0) {
|
||||
int cmp = CellComparator.COMPARATOR.compareRows(firstRowCell, stopRowKey, 0, stopRowKey.length);
|
||||
if (cmp > 0) {
|
||||
done = true;
|
||||
}
|
||||
return done;
|
||||
|
|
|
@ -18,12 +18,15 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -45,6 +48,12 @@ public class LongComparator extends ByteArrayComparable {
|
|||
return this.longValue.compareTo(that);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ByteBuffer value, int offset, int length) {
|
||||
Long that = ByteBufferUtils.toLong(value, offset);
|
||||
return this.longValue.compareTo(that);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The comparator serialized using pb
|
||||
*/
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -88,12 +89,12 @@ public class MultiRowRangeFilter extends FilterBase {
|
|||
// the row key. If index is out of bound which happens when the start row
|
||||
// user sets is after the largest stop row of the ranges, stop the scan.
|
||||
// If row key is after the current range, find the next range and update index.
|
||||
byte[] rowArr = firstRowCell.getRowArray();
|
||||
int length = firstRowCell.getRowLength();
|
||||
int offset = firstRowCell.getRowOffset();
|
||||
if (!initialized
|
||||
|| !range.contains(firstRowCell.getRowArray(), offset, length)) {
|
||||
byte[] rowkey = new byte[length];
|
||||
System.arraycopy(firstRowCell.getRowArray(), firstRowCell.getRowOffset(), rowkey, 0, length);
|
||||
|| !range.contains(rowArr, offset, length)) {
|
||||
byte[] rowkey = CellUtil.cloneRow(firstRowCell);
|
||||
index = getNextRangeIndex(rowkey);
|
||||
if (index >= rangeList.size()) {
|
||||
done = true;
|
||||
|
@ -118,7 +119,7 @@ public class MultiRowRangeFilter extends FilterBase {
|
|||
}
|
||||
initialized = true;
|
||||
} else {
|
||||
if (range.contains(firstRowCell.getRowArray(), offset, length)) {
|
||||
if (range.contains(rowArr, offset, length)) {
|
||||
currentReturnCode = ReturnCode.INCLUDE;
|
||||
} else currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT;
|
||||
}
|
||||
|
|
|
@ -19,12 +19,11 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -72,16 +71,15 @@ public class MultipleColumnPrefixFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell kv) {
|
||||
if (sortedPrefixes.size() == 0 || kv.getQualifierArray() == null) {
|
||||
if (sortedPrefixes.size() == 0) {
|
||||
return ReturnCode.INCLUDE;
|
||||
} else {
|
||||
return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
return filterColumn(kv);
|
||||
}
|
||||
}
|
||||
|
||||
public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) {
|
||||
byte [] qualifier = Arrays.copyOfRange(buffer, qualifierOffset,
|
||||
qualifierLength + qualifierOffset);
|
||||
public ReturnCode filterColumn(Cell cell) {
|
||||
byte [] qualifier = CellUtil.cloneQualifier(cell);
|
||||
TreeSet<byte []> lesserOrEqualPrefixes =
|
||||
(TreeSet<byte []>) sortedPrefixes.headSet(qualifier, true);
|
||||
|
||||
|
@ -163,9 +161,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return KeyValueUtil.createFirstOnRow(
|
||||
cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getFamilyArray(),
|
||||
cell.getFamilyOffset(), cell.getFamilyLength(), hint, 0, hint.length);
|
||||
return CellUtil.createFirstOnRowCol(cell, hint, 0, hint.length);
|
||||
}
|
||||
|
||||
public TreeSet<byte []> createTreeSet() {
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -59,6 +61,11 @@ public class NullComparator extends ByteArrayComparable {
|
|||
return compareTo(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ByteBuffer value, int offset, int length) {
|
||||
return value != null ? 1 : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The comparator serialized using pb
|
||||
*/
|
||||
|
|
|
@ -21,11 +21,13 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -58,8 +60,15 @@ public class PrefixFilter extends FilterBase {
|
|||
// if they are equal, return false => pass row
|
||||
// else return true, filter row
|
||||
// if we are passed the prefix, set flag
|
||||
int cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),
|
||||
this.prefix.length, this.prefix, 0, this.prefix.length);
|
||||
int cmp;
|
||||
if (firstRowCell instanceof ByteBufferedCell) {
|
||||
cmp = ByteBufferUtils.compareTo(((ByteBufferedCell) firstRowCell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) firstRowCell).getRowPositionInByteBuffer(), this.prefix.length,
|
||||
this.prefix, 0, this.prefix.length);
|
||||
} else {
|
||||
cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),
|
||||
this.prefix.length, this.prefix, 0, this.prefix.length);
|
||||
}
|
||||
if ((!isReversed() && cmp > 0) || (isReversed() && cmp < 0)) {
|
||||
passedPrefix = true;
|
||||
}
|
||||
|
|
|
@ -63,8 +63,7 @@ public class QualifierFilter extends CompareFilter {
|
|||
public ReturnCode filterKeyValue(Cell v) {
|
||||
int qualifierLength = v.getQualifierLength();
|
||||
if (qualifierLength > 0) {
|
||||
if (doCompare(this.compareOp, this.comparator, v.getQualifierArray(),
|
||||
v.getQualifierOffset(), qualifierLength)) {
|
||||
if (compareQualifier(this.compareOp, this.comparator, v)) {
|
||||
return ReturnCode.SKIP;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -75,8 +75,7 @@ public class RowFilter extends CompareFilter {
|
|||
|
||||
@Override
|
||||
public boolean filterRowKey(Cell firstRowCell) {
|
||||
if (doCompare(this.compareOp, this.comparator, firstRowCell.getRowArray(),
|
||||
firstRowCell.getRowOffset(), firstRowCell.getRowLength())) {
|
||||
if (compareRow(this.compareOp, this.comparator, firstRowCell)) {
|
||||
this.filterOutRow = true;
|
||||
}
|
||||
return this.filterOutRow;
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -185,17 +186,15 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
foundColumn = true;
|
||||
if (filterColumnValue(c.getValueArray(),
|
||||
c.getValueOffset(), c.getValueLength())) {
|
||||
if (filterColumnValue(c)) {
|
||||
return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
|
||||
}
|
||||
this.matchedColumn = true;
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
private boolean filterColumnValue(final byte [] data, final int offset,
|
||||
final int length) {
|
||||
int compareResult = this.comparator.compareTo(data, offset, length);
|
||||
private boolean filterColumnValue(final Cell cell) {
|
||||
int compareResult = CellComparator.compareValue(cell, this.comparator);
|
||||
switch (this.compareOp) {
|
||||
case LESS:
|
||||
return compareResult <= 0;
|
||||
|
|
|
@ -60,8 +60,7 @@ public class ValueFilter extends CompareFilter {
|
|||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell v) {
|
||||
if (doCompare(this.compareOp, this.comparator, v.getValueArray(),
|
||||
v.getValueOffset(), v.getValueLength())) {
|
||||
if (compareValue(this.compareOp, this.comparator, v)) {
|
||||
return ReturnCode.SKIP;
|
||||
}
|
||||
return ReturnCode.INCLUDE;
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({MiscTests.class, SmallTests.class})
|
||||
public class TestComparators {
|
||||
|
||||
@Test
|
||||
public void testCellFieldsCompare() throws Exception {
|
||||
byte[] r0 = Bytes.toBytes("row0");
|
||||
byte[] r1 = Bytes.toBytes("row1");
|
||||
byte[] r2 = Bytes.toBytes("row2");
|
||||
byte[] f = Bytes.toBytes("cf1");
|
||||
byte[] q1 = Bytes.toBytes("qual1");
|
||||
byte[] q2 = Bytes.toBytes("qual2");
|
||||
byte[] q3 = Bytes.toBytes("r");
|
||||
long l1 = 1234L;
|
||||
byte[] v1 = Bytes.toBytes(l1);
|
||||
long l2 = 2000L;
|
||||
byte[] v2 = Bytes.toBytes(l2);
|
||||
// Row compare
|
||||
KeyValue kv = new KeyValue(r1, f, q1, v1);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
ByteArrayComparable comparable = new BinaryComparator(r1);
|
||||
assertEquals(0, CellComparator.compareRow(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareRow(kv, comparable));
|
||||
kv = new KeyValue(r0, f, q1, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareRow(bbCell, comparable) > 0);
|
||||
assertTrue(CellComparator.compareRow(kv, comparable) > 0);
|
||||
kv = new KeyValue(r2, f, q1, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareRow(bbCell, comparable) < 0);
|
||||
assertTrue(CellComparator.compareRow(kv, comparable) < 0);
|
||||
// Qualifier compare
|
||||
comparable = new BinaryPrefixComparator(Bytes.toBytes("qual"));
|
||||
assertEquals(0, CellComparator.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareQualifier(kv, comparable));
|
||||
kv = new KeyValue(r2, f, q2, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellComparator.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareQualifier(kv, comparable));
|
||||
kv = new KeyValue(r2, f, q3, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareQualifier(bbCell, comparable) < 0);
|
||||
assertTrue(CellComparator.compareQualifier(kv, comparable) < 0);
|
||||
// Value compare
|
||||
comparable = new LongComparator(l1);
|
||||
assertEquals(0, CellComparator.compareValue(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareValue(kv, comparable));
|
||||
kv = new KeyValue(r1, f, q1, v2);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareValue(bbCell, comparable) < 0);
|
||||
assertTrue(CellComparator.compareValue(kv, comparable) < 0);
|
||||
// Family compare
|
||||
comparable = new SubstringComparator("cf");
|
||||
assertEquals(0, CellComparator.compareFamily(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareFamily(kv, comparable));
|
||||
}
|
||||
}
|
|
@ -17,6 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Test;
|
||||
|
@ -35,6 +37,8 @@ public class TestLongComparator {
|
|||
for (int j = 0; j < i; j++) {
|
||||
LongComparator cp = new LongComparator(values[i]);
|
||||
assertEquals(1, cp.compareTo(Bytes.toBytes(values[j])));
|
||||
ByteBuffer data_bb = ByteBuffer.wrap(Bytes.toBytes(values[j]));
|
||||
assertEquals(1, cp.compareTo(data_bb, 0, data_bb.capacity()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -496,6 +497,64 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
return 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare cell's row against given comparator
|
||||
* @param cell
|
||||
* @param comparator
|
||||
* @return result comparing cell's row
|
||||
*/
|
||||
public static int compareRow(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPositionInByteBuffer(), cell.getRowLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare cell's column family against given comparator
|
||||
* @param cell
|
||||
* @param comparator
|
||||
* @return result comparing cell's column family
|
||||
*/
|
||||
public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPositionInByteBuffer(), cell.getFamilyLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare cell's qualifier against given comparator
|
||||
* @param cell
|
||||
* @param comparator
|
||||
* @return result comparing cell's qualifier
|
||||
*/
|
||||
public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPositionInByteBuffer(), cell.getQualifierLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare cell's value against given comparator
|
||||
* @param cell
|
||||
* @param comparator
|
||||
* @return result comparing cell's value
|
||||
*/
|
||||
public static int compareValue(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePositionInByteBuffer(), cell.getValueLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator that compares row component only of a Cell
|
||||
*/
|
||||
|
|
|
@ -18,10 +18,13 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -29,6 +32,10 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
/** Base class for byte array comparators */
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
// TODO Now we are deviating a lot from the actual Comparable<byte[]> what this implements, by
|
||||
// adding special compareTo methods. We have to clean it. Deprecate this class and replace it
|
||||
// with a more generic one which says it compares bytes (not necessary a byte array only)
|
||||
// BytesComparable implements Comparable<Byte> will work?
|
||||
public abstract class ByteArrayComparable implements Comparable<byte[]> {
|
||||
|
||||
byte[] value;
|
||||
|
@ -95,4 +102,21 @@ public abstract class ByteArrayComparable implements Comparable<byte[]> {
|
|||
* is less than, equal to, or greater than the specified object.
|
||||
*/
|
||||
public abstract int compareTo(byte [] value, int offset, int length);
|
||||
|
||||
/**
|
||||
* Special compareTo method for subclasses, to avoid copying bytes unnecessarily.
|
||||
* @param value bytes to compare within a ByteBuffer
|
||||
* @param offset offset into value
|
||||
* @param length number of bytes to compare
|
||||
* @return a negative integer, zero, or a positive integer as this object
|
||||
* is less than, equal to, or greater than the specified object.
|
||||
*/
|
||||
public int compareTo(ByteBuffer value, int offset, int length) {
|
||||
// For BC, providing a default implementation here which is doing a bytes copy to a temp byte[]
|
||||
// and calling compareTo(byte[]). Make sure to override this method in subclasses to avoid
|
||||
// copying bytes unnecessarily.
|
||||
byte[] temp = new byte[length];
|
||||
ByteBufferUtils.copyFromBufferToArray(temp, value, offset, 0, length);
|
||||
return compareTo(temp);
|
||||
}
|
||||
}
|
|
@ -3310,8 +3310,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
matches = true;
|
||||
} else if (result.size() == 1 && !valueIsNull) {
|
||||
Cell kv = result.get(0);
|
||||
int compareResult = comparator.compareTo(kv.getValueArray(),
|
||||
kv.getValueOffset(), kv.getValueLength());
|
||||
int compareResult = CellComparator.compareValue(kv, comparator);
|
||||
switch (compareOp) {
|
||||
case LESS:
|
||||
matches = compareResult < 0;
|
||||
|
@ -3390,8 +3389,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
matches = true;
|
||||
} else if (result.size() == 1 && !valueIsNull) {
|
||||
Cell kv = result.get(0);
|
||||
int compareResult = comparator.compareTo(kv.getValueArray(),
|
||||
kv.getValueOffset(), kv.getValueLength());
|
||||
int compareResult = CellComparator.compareValue(kv, comparator);
|
||||
switch (compareOp) {
|
||||
case LESS:
|
||||
matches = compareResult < 0;
|
||||
|
@ -6840,10 +6838,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
newCell.getQualifierArray(), newCell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
// copy in the value
|
||||
// TODO handle when oldCell is BBBacked
|
||||
System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
|
||||
newCell.getValueArray(), newCell.getValueOffset(),
|
||||
oldCell.getValueLength());
|
||||
CellUtil.copyValueTo(oldCell, newCell.getValueArray(), newCell.getValueOffset());
|
||||
System.arraycopy(cell.getValueArray(), cell.getValueOffset(),
|
||||
newCell.getValueArray(),
|
||||
newCell.getValueOffset() + oldCell.getValueLength(),
|
||||
|
|
|
@ -15,6 +15,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.Test;
|
||||
|
@ -29,15 +31,21 @@ import static org.junit.Assert.assertEquals;
|
|||
public class TestBitComparator {
|
||||
|
||||
private static byte[] zeros = new byte[]{0, 0, 0, 0, 0, 0};
|
||||
private static ByteBuffer zeros_bb = ByteBuffer.wrap(zeros);
|
||||
private static byte[] ones = new byte[]{1, 1, 1, 1, 1, 1};
|
||||
private static ByteBuffer ones_bb = ByteBuffer.wrap(ones);
|
||||
private static byte[] data0 = new byte[]{0, 1, 2, 4, 8, 15};
|
||||
private static byte[] data1 = new byte[]{15, 0, 0, 0, 0, 0};
|
||||
private static ByteBuffer data1_bb = ByteBuffer.wrap(data1);
|
||||
private static byte[] data2 = new byte[]{0, 0, 0, 0, 0, 15};
|
||||
private static ByteBuffer data2_bb = ByteBuffer.wrap(data2);
|
||||
private static byte[] data3 = new byte[]{15, 15, 15, 15, 15};
|
||||
|
||||
// data for testing compareTo method with offset and length parameters
|
||||
private static byte[] data1_2 = new byte[]{15, 15, 0, 0, 0, 0, 0, 15};
|
||||
private static ByteBuffer data1_2_bb = ByteBuffer.wrap(data1_2);
|
||||
private static byte[] data2_2 = new byte[]{15, 0, 0, 0, 0, 0, 15, 15};
|
||||
private static ByteBuffer data2_2_bb = ByteBuffer.wrap(data2_2);
|
||||
|
||||
private final int Equal = 0;
|
||||
private final int NotEqual = 1;
|
||||
|
@ -50,6 +58,13 @@ public class TestBitComparator {
|
|||
testOperation(data2, data1, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
testOperation(ones, data0, BitComparator.BitwiseOp.AND, Equal);
|
||||
testOperation(ones, data3, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
|
||||
testOperation(zeros_bb, ones, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
testOperation(data1_bb, ones, BitComparator.BitwiseOp.AND, Equal);
|
||||
testOperation(data1_bb, data0, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
testOperation(data2_bb, data1, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
testOperation(ones_bb, data0, BitComparator.BitwiseOp.AND, Equal);
|
||||
testOperation(ones_bb, data3, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -59,6 +74,12 @@ public class TestBitComparator {
|
|||
testOperation(data1, zeros, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperation(data2, data1, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperation(ones, data3, BitComparator.BitwiseOp.OR, NotEqual);
|
||||
|
||||
testOperation(ones_bb, zeros, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperation(zeros_bb, zeros, BitComparator.BitwiseOp.OR, NotEqual);
|
||||
testOperation(data1_bb, zeros, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperation(data2_bb, data1, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperation(ones_bb, data3, BitComparator.BitwiseOp.OR, NotEqual);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -68,32 +89,61 @@ public class TestBitComparator {
|
|||
testOperation(ones, ones, BitComparator.BitwiseOp.XOR, NotEqual);
|
||||
testOperation(data2, data1, BitComparator.BitwiseOp.XOR, Equal);
|
||||
testOperation(ones, data3, BitComparator.BitwiseOp.XOR, NotEqual);
|
||||
|
||||
testOperation(ones_bb, zeros, BitComparator.BitwiseOp.XOR, Equal);
|
||||
testOperation(zeros_bb, zeros, BitComparator.BitwiseOp.XOR, NotEqual);
|
||||
testOperation(ones_bb, ones, BitComparator.BitwiseOp.XOR, NotEqual);
|
||||
testOperation(data2_bb, data1, BitComparator.BitwiseOp.XOR, Equal);
|
||||
testOperation(ones_bb, data3, BitComparator.BitwiseOp.XOR, NotEqual);
|
||||
}
|
||||
|
||||
private void testOperation(byte[] data, byte[] comparatorBytes, BitComparator.BitwiseOp operator, int expected) {
|
||||
private void testOperation(byte[] data, byte[] comparatorBytes, BitComparator.BitwiseOp operator,
|
||||
int expected) {
|
||||
BitComparator comparator = new BitComparator(comparatorBytes, operator);
|
||||
assertEquals(comparator.compareTo(data), expected);
|
||||
}
|
||||
|
||||
private void testOperation(ByteBuffer data, byte[] comparatorBytes,
|
||||
BitComparator.BitwiseOp operator, int expected) {
|
||||
BitComparator comparator = new BitComparator(comparatorBytes, operator);
|
||||
assertEquals(comparator.compareTo(data, 0, data.capacity()), expected);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testANDOperationWithOffset() {
|
||||
testOperationWithOffset(data1_2, ones, BitComparator.BitwiseOp.AND, Equal);
|
||||
testOperationWithOffset(data1_2, data0, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
testOperationWithOffset(data2_2, data1, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
|
||||
testOperationWithOffset(data1_2_bb, ones, BitComparator.BitwiseOp.AND, Equal);
|
||||
testOperationWithOffset(data1_2_bb, data0, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
testOperationWithOffset(data2_2_bb, data1, BitComparator.BitwiseOp.AND, NotEqual);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOROperationWithOffset() {
|
||||
testOperationWithOffset(data1_2, zeros, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperationWithOffset(data2_2, data1, BitComparator.BitwiseOp.OR, Equal);
|
||||
|
||||
testOperationWithOffset(data1_2_bb, zeros, BitComparator.BitwiseOp.OR, Equal);
|
||||
testOperationWithOffset(data2_2_bb, data1, BitComparator.BitwiseOp.OR, Equal);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testXOROperationWithOffset() {
|
||||
testOperationWithOffset(data2_2, data1, BitComparator.BitwiseOp.XOR, Equal);
|
||||
|
||||
testOperationWithOffset(data2_2_bb, data1, BitComparator.BitwiseOp.XOR, Equal);
|
||||
}
|
||||
|
||||
private void testOperationWithOffset(byte[] data, byte[] comparatorBytes, BitComparator.BitwiseOp operator, int expected) {
|
||||
private void testOperationWithOffset(byte[] data, byte[] comparatorBytes,
|
||||
BitComparator.BitwiseOp operator, int expected) {
|
||||
BitComparator comparator = new BitComparator(comparatorBytes, operator);
|
||||
assertEquals(comparator.compareTo(data, 1, comparatorBytes.length), expected);
|
||||
}
|
||||
|
||||
private void testOperationWithOffset(ByteBuffer data, byte[] comparatorBytes,
|
||||
BitComparator.BitwiseOp operator, int expected) {
|
||||
BitComparator comparator = new BitComparator(comparatorBytes, operator);
|
||||
assertEquals(comparator.compareTo(data, 1, comparatorBytes.length), expected);
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
|
|
|
@ -22,9 +22,12 @@ import static org.junit.Assert.assertFalse;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -104,39 +107,72 @@ public class TestSingleColumnValueFilter {
|
|||
Bytes.toBytes(1L));
|
||||
assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
Bytes.toBytes(100L));
|
||||
assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
Bytes.toBytes(120L));
|
||||
assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
filter.reset();
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
}
|
||||
|
||||
private void basicFilterTests(SingleColumnValueFilter filter)
|
||||
throws Exception {
|
||||
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
|
||||
assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
|
||||
assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("basicFilterNotNull", filter.filterRow());
|
||||
filter.reset();
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertTrue("basicFilterNotNull", filter.filterRow());
|
||||
filter.reset();
|
||||
filter.setLatestVersionOnly(false);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("basicFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
||||
|
@ -144,10 +180,16 @@ public class TestSingleColumnValueFilter {
|
|||
((SingleColumnValueFilter) filter).setFilterIfMissing(true);
|
||||
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
|
||||
assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("null1FilterRow", filter.filterRow());
|
||||
filter.reset();
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
|
||||
assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertTrue("null2FilterRow", filter.filterRow());
|
||||
}
|
||||
|
||||
|
@ -157,9 +199,15 @@ public class TestSingleColumnValueFilter {
|
|||
FULLSTRING_1);
|
||||
assertTrue("substrTrue",
|
||||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
FULLSTRING_2);
|
||||
assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("substrFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
@ -170,9 +218,15 @@ public class TestSingleColumnValueFilter {
|
|||
FULLSTRING_1);
|
||||
assertTrue("regexTrue",
|
||||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
FULLSTRING_2);
|
||||
assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("regexFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
@ -183,6 +237,9 @@ public class TestSingleColumnValueFilter {
|
|||
FULLSTRING_1);
|
||||
assertTrue("regexTrue",
|
||||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("regexFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue