diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java index 1f5fd95..7c91a7d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java @@ -29,8 +29,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; @@ -137,20 +135,18 @@ public class DependentColumnFilter extends CompareFilter { @Override public ReturnCode filterKeyValue(Cell c) { - // TODO make matching Column a cell method or CellUtil method. - KeyValue v = KeyValueUtil.ensureKeyValue(c); // Check if the column and qualifier match - if (!CellUtil.matchingColumn(v, this.columnFamily, this.columnQualifier)) { + if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) { // include non-matches for the time being, they'll be discarded afterwards return ReturnCode.INCLUDE; } // If it doesn't pass the op, skip it if (comparator != null - && doCompare(compareOp, comparator, v.getValueArray(), v.getValueOffset(), - v.getValueLength())) + && doCompare(compareOp, comparator, c.getValueArray(), c.getValueOffset(), + c.getValueLength())) return ReturnCode.SKIP; - stampSet.add(v.getTimestamp()); + stampSet.add(c.getTimestamp()); if(dropDependentColumn) { return ReturnCode.SKIP; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java index 4988dc8..50a094a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -144,9 +143,7 @@ public abstract class FilterBase extends Filter { * @inheritDoc */ public Cell getNextCellHint(Cell currentKV) throws IOException { - // Old filters based off of this class will override KeyValue getNextKeyHint(KeyValue). - // Thus to maintain compatibility we need to call the old version. - return getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV)); + return null; } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 41c09ce..8e5043e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -26,6 +26,7 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -212,7 +213,11 @@ final public class FilterList extends Filter { @Override public Cell transformCell(Cell v) throws IOException { - return transform(KeyValueUtil.ensureKeyValue(v)); + if (!CellComparator.equals(v, referenceKV)) { + throw new IllegalStateException("Reference Cell: " + this.referenceKV + " does not match: " + + v); + } + return this.transformedKV; } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index 118811d..8c337e3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -24,7 +24,6 @@ import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; @@ -76,15 +75,11 @@ public class FuzzyRowFilter extends FilterBase { // TODO: possible improvement: save which fuzzy row key to use when providing a hint @Override public ReturnCode filterKeyValue(Cell kv) { - // TODO add getRow() equivalent to Cell or change satisfies to take b[],o,l style args. - KeyValue v = KeyValueUtil.ensureKeyValue(kv); - - byte[] rowKey = v.getRow(); // assigning "worst" result first and looking for better options SatisfiesCode bestOption = SatisfiesCode.NO_NEXT; for (Pair fuzzyData : fuzzyKeysData) { - SatisfiesCode satisfiesCode = - satisfies(rowKey, fuzzyData.getFirst(), fuzzyData.getSecond()); + SatisfiesCode satisfiesCode = satisfies(kv.getRowArray(), kv.getRowOffset(), + kv.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); if (satisfiesCode == SatisfiesCode.YES) { return ReturnCode.INCLUDE; } @@ -105,15 +100,12 @@ public class FuzzyRowFilter extends FilterBase { @Override public Cell getNextCellHint(Cell currentKV) { - // TODO make matching Column a cell method or CellUtil method. - KeyValue v = KeyValueUtil.ensureKeyValue(currentKV); - - byte[] rowKey = v.getRow(); byte[] nextRowKey = null; // Searching for the "smallest" row key that satisfies at least one fuzzy row key for (Pair fuzzyData : fuzzyKeysData) { - byte[] nextRowKeyCandidate = getNextForFuzzyRule(rowKey, - fuzzyData.getFirst(), fuzzyData.getSecond()); + byte[] nextRowKeyCandidate = getNextForFuzzyRule(currentKV.getRowArray(), + currentKV.getRowOffset(), currentKV.getRowLength(), fuzzyData.getFirst(), + fuzzyData.getSecond()); if (nextRowKeyCandidate == null) { continue; } @@ -125,10 +117,9 @@ public class FuzzyRowFilter extends FilterBase { if (nextRowKey == null) { // SHOULD NEVER happen // TODO: is there a better way than throw exception? (stop the scanner?) - throw new IllegalStateException("No next row key that satisfies fuzzy exists when" + - " getNextKeyHint() is invoked." + - " Filter: " + this.toString() + - " currentKV: " + currentKV.toString()); + throw new IllegalStateException("No next row key that satisfies fuzzy exists when" + + " getNextKeyHint() is invoked." + " Filter: " + this.toString() + " currentKV: " + + KeyValueUtil.ensureKeyValue(currentKV).toString()); } return KeyValueUtil.createFirstOnRow(nextRowKey); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java index bb36004..58f5f51 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import com.google.common.base.Preconditions; @@ -55,7 +54,6 @@ public class KeyOnlyFilter extends FilterBase { // TODO make matching Column a cell method or CellUtil method. KeyValue v = KeyValueUtil.ensureKeyValue(kv); - return v.createKeyOnly(this.lenAsVal); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java index 1058756..76673bb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java @@ -28,8 +28,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -108,10 +106,9 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { public void filterRowCells(List kvs) { Iterator it = kvs.iterator(); while (it.hasNext()) { - KeyValue kv = KeyValueUtil.ensureKeyValue(it.next()); // If the current column is actually the tested column, // we will skip it instead. - if (CellUtil.matchingColumn(kv, this.columnFamily, this.columnQualifier)) { + if (CellUtil.matchingColumn(it.next(), this.columnFamily, this.columnQualifier)) { it.remove(); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index 34d6549..c59baad 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -28,8 +28,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; @@ -172,9 +170,6 @@ public class SingleColumnValueFilter extends FilterBase { @Override public ReturnCode filterKeyValue(Cell c) { - // TODO get rid of this. - KeyValue keyValue = KeyValueUtil.ensureKeyValue(c); - // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue())); if (this.matchedColumn) { // We already found and matched the single column, all keys now pass @@ -183,12 +178,12 @@ public class SingleColumnValueFilter extends FilterBase { // We found but did not match the single column, skip to next row return ReturnCode.NEXT_ROW; } - if (!CellUtil.matchingColumn(keyValue, this.columnFamily, this.columnQualifier)) { + if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) { return ReturnCode.INCLUDE; } foundColumn = true; - if (filterColumnValue(keyValue.getValueArray(), - keyValue.getValueOffset(), keyValue.getValueLength())) { + if (filterColumnValue(c.getValueArray(), + c.getValueOffset(), c.getValueLength())) { return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE; } this.matchedColumn = true; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java index edf1e52..e9c83cc 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java @@ -353,4 +353,14 @@ public class CellComparator implements Comparator, Serializable{ return 0; } + /** + * Counter part for the KeyValue.RowOnlyComparator + */ + public static class RowComparator extends CellComparator { + @Override + public int compare(Cell a, Cell b) { + return compareRows(a, b); + } + } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java index 18ffd96..4b80fbb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java @@ -25,11 +25,11 @@ import java.util.TreeSet; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.util.Bytes; /** @@ -52,7 +52,7 @@ class GetClosestRowBeforeTracker { private final int tablenamePlusDelimiterLength; // Deletes keyed by row. Comparator compares on row portion of KeyValue only. - private final NavigableMap> deletes; + private final NavigableMap> deletes; /** * @param c @@ -78,7 +78,7 @@ class GetClosestRowBeforeTracker { this.oldestts = System.currentTimeMillis() - ttl; this.kvcomparator = c; KeyValue.RowOnlyComparator rc = new KeyValue.RowOnlyComparator(this.kvcomparator); - this.deletes = new TreeMap>(rc); + this.deletes = new TreeMap>(new CellComparator.RowComparator()); } /** @@ -94,12 +94,12 @@ class GetClosestRowBeforeTracker { * @param kv */ private void addDelete(final Cell kv) { - NavigableSet rowdeletes = this.deletes.get(kv); + NavigableSet rowdeletes = this.deletes.get(kv); if (rowdeletes == null) { - rowdeletes = new TreeSet(this.kvcomparator); - this.deletes.put(KeyValueUtil.ensureKeyValue(kv), rowdeletes); + rowdeletes = new TreeSet(this.kvcomparator); + this.deletes.put(kv, rowdeletes); } - rowdeletes.add(KeyValueUtil.ensureKeyValue(kv)); + rowdeletes.add(kv); } /* @@ -128,7 +128,7 @@ class GetClosestRowBeforeTracker { */ private boolean isDeleted(final Cell kv) { if (this.deletes.isEmpty()) return false; - NavigableSet rowdeletes = this.deletes.get(kv); + NavigableSet rowdeletes = this.deletes.get(kv); if (rowdeletes == null || rowdeletes.isEmpty()) return false; return isDeleted(kv, rowdeletes); } @@ -140,9 +140,9 @@ class GetClosestRowBeforeTracker { * @param ds * @return True is the specified KeyValue is deleted, false if not */ - public boolean isDeleted(final Cell kv, final NavigableSet ds) { + public boolean isDeleted(final Cell kv, final NavigableSet ds) { if (deletes == null || deletes.isEmpty()) return false; - for (KeyValue d: ds) { + for (Cell d: ds) { long kvts = kv.getTimestamp(); long dts = d.getTimestamp(); if (CellUtil.isDeleteFamily(d)) { @@ -164,7 +164,7 @@ class GetClosestRowBeforeTracker { if (kvts > dts) return false; // Check Type - switch (KeyValue.Type.codeToType(d.getType())) { + switch (KeyValue.Type.codeToType(d.getTypeByte())) { case Delete: return kvts == dts; case DeleteColumn: return true; default: continue; @@ -198,7 +198,7 @@ class GetClosestRowBeforeTracker { * @return True if we added a candidate */ boolean handle(final Cell kv) { - if (KeyValueUtil.ensureKeyValue(kv).isDelete()) { + if (CellUtil.isDelete(kv)) { handleDeletes(kv); return false; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 93eada8..683dfcd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -3898,7 +3898,7 @@ public class HRegion implements HeapSize { // , Writable{ if (region != null && region.metricsRegion != null) { long totalSize = 0; for(Cell c:outResults) { - // TODO clean up + // TODO clean up. Find way to remove this ensureKeyValue KeyValue kv = KeyValueUtil.ensureKeyValue(c); totalSize += kv.getLength(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index 0fb3fcd..d504e4b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -1248,14 +1248,14 @@ public class VisibilityController extends BaseRegionObserver implements MasterOb // We need to create another KV, unfortunately, because the current new KV // has no space for tags - KeyValue newKv = KeyValueUtil.ensureKeyValue(newCell); - KeyValue rewriteKv = new KeyValue(newKv.getRowArray(), newKv.getRowOffset(), newKv.getRowLength(), - newKv.getFamilyArray(), newKv.getFamilyOffset(), newKv.getFamilyLength(), - newKv.getQualifierArray(), newKv.getQualifierOffset(), newKv.getQualifierLength(), - newKv.getTimestamp(), KeyValue.Type.codeToType(newKv.getTypeByte()), - newKv.getValueArray(), newKv.getValueOffset(), newKv.getValueLength(), tags); + KeyValue rewriteKv = new KeyValue(newCell.getRowArray(), newCell.getRowOffset(), + newCell.getRowLength(), newCell.getFamilyArray(), newCell.getFamilyOffset(), + newCell.getFamilyLength(), newCell.getQualifierArray(), newCell.getQualifierOffset(), + newCell.getQualifierLength(), newCell.getTimestamp(), KeyValue.Type.codeToType(newCell + .getTypeByte()), newCell.getValueArray(), newCell.getValueOffset(), + newCell.getValueLength(), tags); // Preserve mvcc data - rewriteKv.setSequenceId(newKv.getMvccVersion()); + rewriteKv.setSequenceId(newCell.getMvccVersion()); return rewriteKv; }