diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java index 665c59c..82e3786 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -124,7 +125,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C * @param qualifier * @param ts * @param value - * @param tags - Specify the Tags as an Array {@link KeyValue.Tag} + * @param tags - Specify the Tags as an Array * @return a KeyValue with this objects row key and the Put identifier. */ KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts, byte[] value, Tag[] tags) { @@ -138,7 +139,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C * @return a KeyValue with this objects row key and the Put identifier. */ KeyValue createPutKeyValue(byte[] family, ByteBuffer qualifier, long ts, ByteBuffer value, - Tag[] tags) { + Tag[] tags) { return new KeyValue(this.row, 0, this.row == null ? 0 : this.row.length, family, 0, family == null ? 0 : family.length, qualifier, ts, KeyValue.Type.Put, value, tags != null ? Arrays.asList(tags) : null); @@ -215,15 +216,17 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C private static Map cellToStringMap(Cell c) { Map stringMap = new HashMap(); + assert c.hasArray(); stringMap.put("qualifier", Bytes.toStringBinary(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength())); stringMap.put("timestamp", c.getTimestamp()); stringMap.put("vlen", c.getValueLength()); - List tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + List tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); if (tags != null) { List tagsString = new ArrayList(); for (Tag t : tags) { - tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(t.getValue())); + tagsString.add((t.getType()) + ":" + + Bytes.toStringBinary(t.getBytes(), t.getTagOffset(), t.getTagLength())); } stringMap.put("tag", tagsString); } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java index 0fc624f..36381c4 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java @@ -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; @@ -47,7 +50,12 @@ public class BinaryComparator extends ByteArrayComparable { public int compareTo(byte [] value, int offset, int length) { 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(this.value, 0, this.value.length, value, offset, length); + } + /** * @return The comparator serialized using pb */ diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java index b6311b0..1bf582b 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java @@ -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,12 @@ public class BinaryPrefixComparator extends ByteArrayComparable { this.value.length <= length ? this.value.length : length); } + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + return ByteBufferUtils.compareTo(this.value, 0, this.value.length, value, offset, + this.value.length <= length ? this.value.length : length); + } + /** * @return The comparator serialized using pb */ diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java index d527a2c..334111b 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java @@ -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; + } } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java index 4642ab9..c0fa23a 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java @@ -18,6 +18,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; @@ -37,7 +39,7 @@ public abstract class ByteArrayComparable implements Comparable { * Constructor. * @param value the value to compare against */ - public ByteArrayComparable(byte [] value) { + public ByteArrayComparable(byte[] value) { this.value = value; } @@ -95,4 +97,7 @@ public abstract class ByteArrayComparable implements Comparable { * is less than, equal to, or greater than the specified object. */ public abstract int compareTo(byte [] value, int offset, int length); + + // TODO a BB based method does not fit into this class ByteArrayComparable? + public abstract int compareTo(ByteBuffer value, int offset, int length); } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java index d8ea094..999d771 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java @@ -21,14 +21,17 @@ package org.apache.hadoop.hbase.filter; import static org.apache.hadoop.hbase.util.Bytes.len; +import java.nio.ByteBuffer; import java.util.ArrayList; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.FirstOnColumnCell; import org.apache.hadoop.hbase.KeyValueUtil; 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; @@ -118,14 +121,19 @@ public class ColumnRangeFilter extends FilterBase { @Override public ReturnCode filterKeyValue(Cell kv) { // TODO have a column compare method in Cell - byte[] buffer = kv.getQualifierArray(); + ByteBuffer buffer = kv.getQualifierBuffer(); 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); + if (kv.hasArray()) { + cmpMin = Bytes.compareTo(kv.getQualifierArray(), qualifierOffset, qualifierLength, + this.minColumn, 0, this.minColumn.length); + } else { + cmpMin = ByteBufferUtils.compareTo(kv.getQualifierBuffer(), qualifierOffset, + qualifierLength, this.minColumn, 0, this.minColumn.length); + } } if (cmpMin < 0) { @@ -140,8 +148,14 @@ public class ColumnRangeFilter extends FilterBase { return ReturnCode.INCLUDE; } - int cmpMax = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, - this.maxColumn, 0, this.maxColumn.length); + int cmpMax; + if (kv.hasArray()) { + cmpMax = Bytes.compareTo(kv.getQualifierArray(), qualifierOffset, qualifierLength, + this.maxColumn, 0, this.maxColumn.length); + } else { + cmpMax = ByteBufferUtils.compareTo(kv.getQualifierBuffer(), qualifierOffset, qualifierLength, + this.maxColumn, 0, this.maxColumn.length); + } if (this.maxColumnInclusive && cmpMax <= 0 || !this.maxColumnInclusive && cmpMax < 0) { @@ -217,10 +231,14 @@ 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)); - + if (cell.hasArray()) { + return KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), + cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), + cell.getFamilyLength(), this.minColumn, 0, len(this.minColumn)); + } + return new FirstOnColumnCell(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength(), + cell.getFamilyBuffer(), cell.getFamilyOffset(), cell.getFamilyLength(), + ByteBuffer.wrap(this.minColumn), 0, len(this.minColumn)); } @Override diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java index 319e123..ed3a0e3 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.nio.ByteBuffer; import java.util.ArrayList; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -122,6 +123,32 @@ public abstract class CompareFilter extends FilterBase { compareOp.name()); } } + + protected boolean doCompare(final CompareOp compareOp, + final ByteArrayComparable comparator, final ByteBuffer data, + final int offset, final int length) { + if (compareOp == CompareOp.NO_OP) { + return true; + } + int compareResult = comparator.compareTo(data, offset, length); + 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()); + } + } // returns an array of heterogeneous objects public static ArrayList extractArguments(ArrayList filterArguments) { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java index 6d19842..ac6da83 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java @@ -142,7 +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(), + && doCompare(compareOp, comparator, c.getValueBuffer(), c.getValueOffset(), c.getValueLength())) return ReturnCode.SKIP; diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java index e79a4d5..a7a5a29 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java @@ -63,9 +63,16 @@ 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)) { - return ReturnCode.NEXT_ROW; + if (v.hasArray()) { + if (doCompare(this.compareOp, this.comparator, v.getFamilyArray(), v.getFamilyOffset(), + familyLength)) { + return ReturnCode.NEXT_ROW; + } + } else { + if (doCompare(this.compareOp, this.comparator, v.getFamilyBuffer(), v.getFamilyOffset(), + familyLength)) { + return ReturnCode.NEXT_ROW; + } } } return ReturnCode.INCLUDE; diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java index 88bf842..72f28ce 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java @@ -20,12 +20,14 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; 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.util.ByteRange; /** * Interface for row and column filters directly applied within the regionserver. @@ -78,8 +80,25 @@ public abstract class Filter { * @param length length of the row key * @return true, remove entire row, false, include the row (maybe). * @throws IOException in case an I/O or an filter specific failure needs to be signaled. + * @deprecated Instead use {@link #filterRowKey(ByteBuffer, int, int)} */ + @Deprecated abstract public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException; + + /** + * Filters a row based on the row key. If this returns true, the entire row will be excluded. If + * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below. + * + * Concrete implementers can signal a failure condition in their code by throwing an + * {@link IOException}. + * + * @param buffer buffer containing row key + * @param offset offset into buffer where row key starts + * @param length length of the row key + * @return true, remove entire row, false, include the row (maybe). + * @throws IOException in case an I/O or an filter specific failure needs to be signaled. + */ + abstract public boolean filterRowKey(ByteBuffer buffer, int offset, int length) throws IOException; /** * If this returns true, the scan will terminate. diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java index a04dd89..5b5c8ec 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java @@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.ByteBufferUtils; /** * Abstract base class to help you implement new Filters. Common "ignore" or NOOP type @@ -53,10 +55,22 @@ public abstract class FilterBase extends Filter { * * @inheritDoc */ + @Deprecated @Override public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { return false; } + + @Override + public boolean filterRowKey(ByteBuffer buffer, int offset, int length) throws IOException { + // fall back to old method + if (buffer.hasArray()) { + return filterRowKey(buffer.array(), buffer.arrayOffset() + offset, length); + } + byte[] temp = new byte[length]; + ByteBufferUtils.copyFromBufferToByteArray(temp, buffer, offset, 0, length); + return filterRowKey(temp, 0, length); + } /** * Filters that never filter all remaining can inherit this implementation that diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index ba1a818..37db19e 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.util.ByteRange; import com.google.protobuf.InvalidProtocolBufferException; @@ -196,6 +198,27 @@ final public class FilterList extends Filter { } return flag; } + + @Override + public boolean filterRowKey(ByteBuffer rowKey, int offset, int length) throws IOException { + boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false; + int listize = filters.size(); + for (int i = 0; i < listize; i++) { + Filter filter = filters.get(i); + if (this.operator == Operator.MUST_PASS_ALL) { + if (filter.filterAllRemaining() || + filter.filterRowKey(rowKey, offset, length)) { + flag = true; + } + } else if (this.operator == Operator.MUST_PASS_ONE) { + if (!filter.filterAllRemaining() && + !filter.filterRowKey(rowKey, offset, length)) { + flag = false; + } + } + } + return flag; + } @Override public boolean filterAllRemaining() throws IOException { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java index 5176115..4b971a5 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import org.apache.hadoop.hbase.Cell; @@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.util.ByteRange; import com.google.protobuf.InvalidProtocolBufferException; @@ -104,6 +106,11 @@ final public class FilterWrapper extends Filter { public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { return this.filter.filterRowKey(buffer, offset, length); } + + @Override + public boolean filterRowKey(ByteBuffer buffer, int offset, int length) throws IOException { + return this.filter.filterRowKey(buffer, offset, length); + } @Override public ReturnCode filterKeyValue(Cell v) throws IOException { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index 9b99b71..6457faa 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -89,8 +90,14 @@ public class FuzzyRowFilter extends FilterBase { // assigning "worst" result first and looking for better options SatisfiesCode bestOption = SatisfiesCode.NO_NEXT; for (Pair fuzzyData : fuzzyKeysData) { - SatisfiesCode satisfiesCode = satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), - c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); + SatisfiesCode satisfiesCode; + if (c.hasArray()) { + satisfiesCode = satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), + c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); + } else { + satisfiesCode = satisfies(isReversed(), c.getRowBuffer(), c.getRowOffset(), + c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); + } if (satisfiesCode == SatisfiesCode.YES) { return ReturnCode.INCLUDE; } @@ -114,9 +121,16 @@ public class FuzzyRowFilter extends FilterBase { byte[] nextRowKey = null; // Searching for the "smallest" row key that satisfies at least one fuzzy row key for (Pair fuzzyData : fuzzyKeysData) { - byte[] nextRowKeyCandidate = getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), - currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(), - fuzzyData.getSecond()); + byte[] nextRowKeyCandidate; + if (currentCell.hasArray()) { + nextRowKeyCandidate = getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), + currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(), + fuzzyData.getSecond()); + } else { + nextRowKeyCandidate = getNextForFuzzyRule(isReversed(), currentCell.getRowBuffer(), + currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(), + fuzzyData.getSecond()); + } if (nextRowKeyCandidate == null) { continue; } @@ -266,6 +280,61 @@ public class FuzzyRowFilter extends FilterBase { return SatisfiesCode.YES; } + + private static SatisfiesCode satisfies(boolean reverse, ByteBuffer row, int offset, int length, + byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + if (row == null) { + // do nothing, let scan to proceed + return SatisfiesCode.YES; + } + + Order order = Order.orderFor(reverse); + boolean nextRowKeyCandidateExists = false; + + for (int i = 0; i < fuzzyKeyMeta.length && i < length; i++) { + // First, checking if this position is fixed and not equals the given one + boolean byteAtPositionFixed = fuzzyKeyMeta[i] == 0; + boolean fixedByteIncorrect = byteAtPositionFixed && fuzzyKeyBytes[i] != row.get(i + offset); + if (fixedByteIncorrect) { + // in this case there's another row that satisfies fuzzy rule and bigger + // than this row + if (nextRowKeyCandidateExists) { + return SatisfiesCode.NEXT_EXISTS; + } + + // If this row byte is less than fixed then there's a byte array bigger + // than + // this row and which satisfies the fuzzy rule. Otherwise there's no + // such byte array: + // this row is simply bigger than any byte array that satisfies the + // fuzzy rule + boolean rowByteLessThanFixed = (row.get(i + offset) & 0xFF) < (fuzzyKeyBytes[i] & 0xFF); + if (rowByteLessThanFixed && !reverse) { + return SatisfiesCode.NEXT_EXISTS; + } else if (!rowByteLessThanFixed && reverse) { + return SatisfiesCode.NEXT_EXISTS; + } else { + return SatisfiesCode.NO_NEXT; + } + } + + // Second, checking if this position is not fixed and byte value is not + // the biggest. In this + // case there's a byte array bigger than this row and which satisfies the + // fuzzy rule. To get + // bigger byte array that satisfies the rule we need to just increase this + // byte + // (see the code of getNextForFuzzyRule below) by one. + // Note: if non-fixed byte is already at biggest value, this doesn't allow + // us to say there's + // bigger one that satisfies the rule as it can't be increased. + if (fuzzyKeyMeta[i] == 1 && !order.isMax(fuzzyKeyBytes[i])) { + nextRowKeyCandidateExists = true; + } + } + + return SatisfiesCode.YES; + } @VisibleForTesting static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { @@ -400,6 +469,74 @@ public class FuzzyRowFilter extends FilterBase { return result; } + + /** + * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, + * null otherwise + */ + private static byte[] getNextForFuzzyRule(boolean reverse, ByteBuffer row, int offset, int length, + byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + // To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than + // the given one we do the following: + // 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes + // 2. if during the first step given row did not increase, then we increase the value at + // the first "non-fixed" position (where it is not maximum already) + + // It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position + // values than otherwise. + byte[] result = Arrays.copyOf(fuzzyKeyBytes, + length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length); + if (reverse && length > fuzzyKeyBytes.length) { + // we need trailing 0xff's instead of trailing 0x00's + for (int i = fuzzyKeyBytes.length; i < result.length; i++) { + result[i] = (byte) 0xFF; + } + } + int toInc = -1; + final Order order = Order.orderFor(reverse); + + boolean increased = false; + for (int i = 0; i < result.length; i++) { + if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) { + result[i] = row.get(offset + i); + if (!order.isMax(row.get(i))) { + // this is "non-fixed" position and is not at max value, hence we can increase it + toInc = i; + } + } else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == 0) { + if (order.lt((row.get(i + offset) & 0xFF), (fuzzyKeyBytes[i] & 0xFF))) { + // if setting value for any fixed position increased the original array, + // we are OK + increased = true; + break; + } + + if (order.gt((row.get(i + offset) & 0xFF), (fuzzyKeyBytes[i] & 0xFF))) { + // if setting value for any fixed position makes array "smaller", then just stop: + // in case we found some non-fixed position to increase we will do it, otherwise + // there's no "next" row key that satisfies fuzzy rule and "greater" than given row + break; + } + } + } + + if (!increased) { + if (toInc < 0) { + return null; + } + result[toInc] = order.inc(result[toInc]); + + // Setting all "non-fixed" positions to zeroes to the right of the one we increased so + // that found "next" row key is the smallest possible + for (int i = toInc + 1; i < result.length; i++) { + if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) { + result[i] = order.min(); + } + } + } + + return result; + } /** * @return true if and only if the fields of the filter that are serialized diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java index cf2d153..6e954ae 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.nio.ByteBuffer; import java.util.ArrayList; import org.apache.hadoop.hbase.Cell; @@ -26,6 +27,7 @@ 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,7 +60,8 @@ public class InclusiveStopFilter extends FilterBase { return ReturnCode.INCLUDE; } - public boolean filterRowKey(byte[] buffer, int offset, int length) { + @Override + public boolean filterRowKey(ByteBuffer buffer, int offset, int length) { if (buffer == null) { //noinspection RedundantIfStatement if (this.stopRowKey == null) { @@ -67,7 +70,7 @@ public class InclusiveStopFilter extends FilterBase { return false; } // if stopRowKey is <= buffer, then true, filter row. - int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length, + int cmp = ByteBufferUtils.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length); if(cmp < 0) { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java index 91eef6a..2c0af74 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/LongComparator.java @@ -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; /** @@ -44,6 +47,12 @@ public class LongComparator extends ByteArrayComparable { Long that = Bytes.toLong(value, offset, length); return this.longValue.compareTo(that); } + + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + Long that = ByteBufferUtils.getAsLong(value, offset, length); + return this.longValue.compareTo(that); + } /** * @return The comparator serialized using pb diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java index a9b3c8e..bacf373 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java @@ -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.ByteRange; import com.google.protobuf.InvalidProtocolBufferException; @@ -42,7 +45,7 @@ public class NullComparator extends ByteArrayComparable { public int compareTo(byte[] value) { return value != null ? 1 : 0; } - + @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings (value="EQ_UNUSUAL", justification="") public boolean equals(Object obj) { @@ -58,6 +61,11 @@ public class NullComparator extends ByteArrayComparable { public int compareTo(byte[] value, int offset, int length) { return compareTo(value); } + + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + return value != null ? 1 : 0; + } /** * @return The comparator serialized using pb diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java index 5b56748..bc85ead 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.filter; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import org.apache.hadoop.hbase.Cell; @@ -26,6 +28,7 @@ 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; @@ -50,15 +53,14 @@ public class PrefixFilter extends FilterBase { return prefix; } - public boolean filterRowKey(byte[] buffer, int offset, int length) { - if (buffer == null || this.prefix == null) - return true; - if (length < prefix.length) - return true; + @Override + public boolean filterRowKey(ByteBuffer buffer, int offset, int length) throws IOException { + if (buffer == null || this.prefix == null) return true; + if (length < prefix.length) return true; // 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(buffer, offset, this.prefix.length, this.prefix, 0, + int cmp = ByteBufferUtils.compareTo(buffer, offset, this.prefix.length, this.prefix, 0, this.prefix.length); if ((!isReversed() && cmp > 0) || (isReversed() && cmp < 0)) { passedPrefix = true; @@ -66,7 +68,7 @@ public class PrefixFilter extends FilterBase { filterRow = (cmp != 0); return filterRow; } - + @Override public ReturnCode filterKeyValue(Cell v) { if (filterRow) return ReturnCode.NEXT_ROW; diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java index fb183f1..9c26e12 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java @@ -63,9 +63,16 @@ 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)) { - return ReturnCode.SKIP; + if (v.hasArray()) { + if (doCompare(this.compareOp, this.comparator, v.getQualifierArray(), + v.getQualifierOffset(), qualifierLength)) { + return ReturnCode.SKIP; + } + } else { + if (doCompare(this.compareOp, this.comparator, v.getQualifierBuffer(), + v.getQualifierOffset(), qualifierLength)) { + return ReturnCode.SKIP; + } } } return ReturnCode.INCLUDE; diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java index 70dd1f9..237ee41 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.nio.charset.IllegalCharsetNameException; import java.util.Arrays; @@ -29,6 +30,7 @@ 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 org.jcodings.Encoding; import org.jcodings.EncodingDB; @@ -151,6 +153,11 @@ public class RegexStringComparator extends ByteArrayComparable { public int compareTo(byte[] value, int offset, int length) { return engine.compareTo(value, offset, length); } + + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + return engine.compareTo(value, offset, length); + } /** * @return The comparator serialized using pb @@ -255,6 +262,8 @@ public class RegexStringComparator extends ByteArrayComparable { * @return 0 if a match was made, 1 otherwise */ int compareTo(byte[] value, int offset, int length); + + int compareTo(ByteBuffer value, int offset, int length); } /** @@ -304,6 +313,20 @@ public class RegexStringComparator extends ByteArrayComparable { } return pattern.matcher(tmp).find() ? 0 : 1; } + + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + // Use find() for subsequence match instead of matches() (full sequence + // match) to adhere to the principle of least surprise. + String tmp; + // No optimization done here.. always take copy the byte[] from the BB + // We don have charset accepting String that could be formed from + // ByteBufferUtils.toStringBinary + byte[] val = new byte[length]; + ByteBufferUtils.copyFromBufferToByteArray(val, value, offset, 0, length); + tmp = new String(val, charset); + return pattern.matcher(tmp).find() ? 0 : 1; + } @Override public byte[] toByteArray() { @@ -364,6 +387,16 @@ public class RegexStringComparator extends ByteArrayComparable { Matcher m = pattern.matcher(value); return m.search(offset, length, pattern.getOptions()) < 0 ? 1 : 0; } + + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + // Use subsequence match instead of full sequence match to adhere to the + // principle of least surprise. + byte[] val = new byte[length]; + ByteBufferUtils.copyFromBufferToByteArray(val, value, offset, 0, length); + Matcher m = pattern.matcher(val); + return m.search(offset, length, pattern.getOptions()) < 0 ? 1 : 0; + } @Override public byte[] toByteArray() { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java index cb4337e..a137dec 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import org.apache.hadoop.hbase.Cell; @@ -74,7 +75,7 @@ public class RowFilter extends CompareFilter { } @Override - public boolean filterRowKey(byte[] data, int offset, int length) { + public boolean filterRowKey(ByteBuffer data, int offset, int length) throws IOException { if(doCompare(this.compareOp, this.comparator, data, offset, length)) { this.filterOutRow = true; } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index d905868..b0ca0b7 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -170,7 +170,6 @@ public class SingleColumnValueFilter extends FilterBase { @Override public ReturnCode filterKeyValue(Cell 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 return ReturnCode.INCLUDE; @@ -182,17 +181,22 @@ 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 c) { + int compareResult; + if (c.hasArray()) { + compareResult = this.comparator.compareTo(c.getValueArray(), c.getValueOffset(), + c.getValueLength()); + } else { + compareResult = this.comparator.compareTo(c.getValueBuffer(), c.getValueOffset(), + c.getValueLength()); + } switch (this.compareOp) { case LESS: return compareResult <= 0; diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java index 63fd0a3..9ba5973 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java @@ -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.Bytes; import com.google.protobuf.InvalidProtocolBufferException; @@ -67,6 +70,12 @@ public class SubstringComparator extends ByteArrayComparable { return Bytes.toString(value, offset, length).toLowerCase().contains(substr) ? 0 : 1; } + + @Override + public int compareTo(ByteBuffer value, int offset, int length) { + return ByteBufferUtils.toStringBinary(value, offset, length).toLowerCase().contains(substr) ? 0 + :1; + } /** * @return The comparator serialized using pb diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java index a2c5eb2..5be8acd 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java @@ -60,9 +60,16 @@ public class ValueFilter extends CompareFilter { @Override public ReturnCode filterKeyValue(Cell v) { - if (doCompare(this.compareOp, this.comparator, v.getValueArray(), - v.getValueOffset(), v.getValueLength())) { - return ReturnCode.SKIP; + if (v.hasArray()) { + if (doCompare(this.compareOp, this.comparator, v.getValueArray(), v.getValueOffset(), + v.getValueLength())) { + return ReturnCode.SKIP; + } + } else { + if (doCompare(this.compareOp, this.comparator, v.getValueBuffer(), v.getValueOffset(), + v.getValueLength())) { + return ReturnCode.SKIP; + } } return ReturnCode.INCLUDE; } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java index 31d4f77..7634638 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -71,6 +72,13 @@ public class WhileMatchFilter extends FilterBase { changeFAR(value); return value; } + + @Override + public boolean filterRowKey(ByteBuffer buffer, int offset, int length) throws IOException { + boolean value = filter.filterRowKey(buffer, offset, length); + changeFAR(value); + return value; + } @Override public ReturnCode filterKeyValue(Cell v) throws IOException { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java index e55a7eb..8520449 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java @@ -322,7 +322,8 @@ public class AsyncRpcClient extends AbstractRpcClient { * @throws java.io.IOException if block creation fails */ public ByteBuffer buildCellBlock(CellScanner cells) throws IOException { - return ipcUtil.buildCellBlock(this.codec, this.compressor, cells); + // TODO : check in asyncRPCClient about the offheap nature + return ipcUtil.buildCellBlock(this.codec, this.compressor, cells, false); } /** diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java index 7c6c9ba..50e579c 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java @@ -91,9 +91,8 @@ public class IPCUtil { */ @SuppressWarnings("resource") public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor, - final CellScanner cellScanner) - throws IOException { - return buildCellBlock(codec, compressor, cellScanner, null); + final CellScanner cellScanner, final boolean offheap) throws IOException { + return buildCellBlock(codec, compressor, cellScanner, offheap, null); } /** @@ -113,7 +112,7 @@ public class IPCUtil { */ @SuppressWarnings("resource") public ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor, - final CellScanner cellScanner, final BoundedByteBufferPool pool) + final CellScanner cellScanner, final boolean offheap, final BoundedByteBufferPool pool) throws IOException { if (cellScanner == null) return null; if (codec == null) throw new CellScannerButNoCodecException(); @@ -126,20 +125,21 @@ public class IPCUtil { } else { // Then we need to make our own to return. if (cellScanner instanceof HeapSize) { - long longSize = ((HeapSize)cellScanner).heapSize(); + long longSize = ((HeapSize) cellScanner).heapSize(); // Just make sure we don't have a size bigger than an int. if (longSize > Integer.MAX_VALUE) { throw new IOException("Size " + longSize + " > " + Integer.MAX_VALUE); } bufferSize = ClassSize.align((int)longSize); } - baos = new ByteBufferOutputStream(bufferSize); + baos = new ByteBufferOutputStream(bufferSize, offheap); } OutputStream os = baos; Compressor poolCompressor = null; try { if (compressor != null) { - if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf); + if (compressor instanceof Configurable) + ((Configurable) compressor).setConf(this.conf); poolCompressor = CodecPool.getCompressor(compressor); os = compressor.createOutputStream(os, poolCompressor); } @@ -150,22 +150,24 @@ public class IPCUtil { count++; } encoder.flush(); - // If no cells, don't mess around. Just return null (could be a bunch of existence checking + // If no cells, don't mess around. Just return null (could be a bunch of + // existence checking // gets or something -- stuff that does not return a cell). - if (count == 0) return null; + if (count == 0) + return null; } finally { os.close(); - if (poolCompressor != null) CodecPool.returnCompressor(poolCompressor); + if (poolCompressor != null) + CodecPool.returnCompressor(poolCompressor); } if (LOG.isTraceEnabled()) { if (bufferSize < baos.size()) { - LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size() + - "; up hbase.ipc.cellblock.building.initial.buffersize?"); + LOG.trace("Buffer grew from initial bufferSize=" + bufferSize + " to " + baos.size() + + "; up hbase.ipc.cellblock.building.initial.buffersize?"); } } return baos.getByteBuffer(); } - /** * @param codec * @param cellBlock diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java index 7492474..3e6645a 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java @@ -870,7 +870,7 @@ public class RpcClientImpl extends AbstractRpcClient { } builder.setMethodName(call.md.getName()); builder.setRequestParam(call.param != null); - ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells); + ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells,false); if (cellBlock != null) { CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder(); cellBlockBuilder.setLength(cellBlock.limit()); diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index bad3cb4..9a2985e 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Consistency; @@ -582,11 +583,9 @@ public final class ProtobufUtil { byte[] tags; if (qv.hasTags()) { tags = qv.getTags().toByteArray(); - Object[] array = Tag.asList(tags, 0, (short)tags.length).toArray(); - Tag[] tagArray = new Tag[array.length]; - for(int i = 0; i< array.length; i++) { - tagArray[i] = (Tag)array[i]; - } + List tagsLst = TagUtil.asList(tags, 0, (short) tags.length); + Tag[] tagArray = new Tag[tagsLst.size()]; + tagsLst.toArray(tagArray); if(qv.hasDeleteType()) { byte[] qual = qv.hasQualifier() ? qv.getQualifier().toByteArray() : null; put.add(new KeyValue(proto.getRow().toByteArray(), family, qual, ts, @@ -2477,16 +2476,32 @@ public final class ProtobufUtil { // Doing this is going to kill us if we do it for all data passed. // St.Ack 20121205 CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder(); - kvbuilder.setRow(ByteStringer.wrap(kv.getRowArray(), kv.getRowOffset(), + ByteBuffer rb = kv.getRowBuffer().duplicate(); + rb.position(kv.getRowOffset()); + kvbuilder.setRow(ByteString.copyFrom(rb, kv.getRowLength())); + ByteBuffer fb = kv.getFamilyBuffer().duplicate(); + fb.position(kv.getFamilyOffset()); + kvbuilder.setFamily(ByteString.copyFrom(fb, kv.getFamilyLength())); + ByteBuffer qb = kv.getQualifierBuffer().duplicate(); + qb.position(kv.getQualifierOffset()); + kvbuilder.setQualifier(ByteString.copyFrom(qb, kv.getQualifierLength())); + // TODO when the buffers having hasArray() go with wrap() + + /*kvbuilder.setRow(ByteStringer.wrap(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength())); kvbuilder.setFamily(ByteStringer.wrap(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength())); kvbuilder.setQualifier(ByteStringer.wrap(kv.getQualifierArray(), - kv.getQualifierOffset(), kv.getQualifierLength())); + kv.getQualifierOffset(), kv.getQualifierLength()));*/ kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); kvbuilder.setTimestamp(kv.getTimestamp()); - kvbuilder.setValue(ByteStringer.wrap(kv.getValueArray(), kv.getValueOffset(), - kv.getValueLength())); + + ByteBuffer vb = kv.getValueBuffer().duplicate(); + vb.position(kv.getValueOffset()); + kvbuilder.setValue(ByteString.copyFrom(vb, kv.getValueLength())); + + /*kvbuilder.setValue(ByteStringer.wrap(kv.getValueArray(), kv.getValueOffset(), + kv.getValueLength()));*/ return kvbuilder.build(); } diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java index 3eab225..57fdcfc 100644 --- hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java +++ hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java @@ -77,8 +77,8 @@ public class TestIPCUtil { throws IOException { Cell [] cells = getCells(count, size); CellScanner cellScanner = sized? getSizedCellScanner(cells): - CellUtil.createCellScanner(Arrays.asList(cells).iterator()); - ByteBuffer bb = util.buildCellBlock(codec, compressor, cellScanner); + CellUtil.createCellScanner(Arrays.asList(cells)); + ByteBuffer bb = util.buildCellBlock(codec, compressor, cellScanner, false); cellScanner = util.createCellScanner(codec, compressor, bb.array(), 0, bb.limit()); int i = 0; while (cellScanner.advance()) { diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java index e6d6f43..a9481c0 100644 --- hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java +++ hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -73,6 +74,10 @@ public class TestPayloadCarryingRpcController { private final int i = index; @Override + public boolean hasArray() { + return true; + } + @Override public byte[] getRowArray() { // unused return null; @@ -206,6 +211,26 @@ public class TestPayloadCarryingRpcController { // unused return null; } + @Override + public ByteBuffer getRowBuffer() { + return null; + } + @Override + public ByteBuffer getFamilyBuffer() { + return null; + } + @Override + public ByteBuffer getQualifierBuffer() { + return null; + } + @Override + public ByteBuffer getValueBuffer() { + return null; + } + @Override + public ByteBuffer getTagsBuffer() { + return null; + } }; } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java new file mode 100644 index 0000000..725c145 --- /dev/null +++ hbase-common/src/main/java/org/apache/hadoop/hbase/ArrayBackedTag.java @@ -0,0 +1,179 @@ +/** + * Copyright The Apache Software Foundation + * + * 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; + +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Tags are part of cells and helps to add metadata about the KVs. + * Metadata could be ACLs per cells, visibility labels, etc. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class ArrayBackedTag implements Tag { + private final byte type; + private final byte[] bytes; + private int offset = 0; + private int length = 0; + + // The special tag will write the length of each tag and that will be + // followed by the type and then the actual tag. + // So every time the length part is parsed we need to add + 1 byte to it to + // get the type and then get the actual tag. + public ArrayBackedTag(byte tagType, String tag) { + this(tagType, Bytes.toBytes(tag)); + } + + /** + * @param tagType + * @param tag + */ + public ArrayBackedTag(byte tagType, byte[] tag) { + /** + * Format for a tag : taglength is serialized + * using 2 bytes only but as this will be unsigned, we can have max taglength of + * (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length. + */ + int tagLength = tag.length + TYPE_LENGTH_SIZE; + if (tagLength > MAX_TAG_LENGTH) { + throw new IllegalArgumentException( + "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH); + } + length = TAG_LENGTH_SIZE + tagLength; + bytes = new byte[length]; + int pos = Bytes.putAsShort(bytes, 0, tagLength); + pos = Bytes.putByte(bytes, pos, tagType); + Bytes.putBytes(bytes, pos, tag, 0, tag.length); + this.type = tagType; + } + + /** + * Creates a Tag from the specified byte array and offset. Presumes + * bytes content starting at offset is formatted as + * a Tag blob. + * The bytes to include the tag type, tag length and actual tag bytes. + * @param bytes + * byte array + * @param offset + * offset to start of Tag + */ + public ArrayBackedTag(byte[] bytes, int offset) { + this(bytes, offset, getLength(bytes, offset)); + } + + private static int getLength(byte[] bytes, int offset) { + return TAG_LENGTH_SIZE + Bytes.readAsInt(bytes, offset, TAG_LENGTH_SIZE); + } + + /** + * Creates a Tag from the specified byte array, starting at offset, and for length + * length. Presumes bytes content starting at offset is + * formatted as a Tag blob. + * @param bytes + * byte array + * @param offset + * offset to start of the Tag + * @param length + * length of the Tag + */ + public ArrayBackedTag(byte[] bytes, int offset, int length) { + if (length > MAX_TAG_LENGTH) { + throw new IllegalArgumentException( + "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH); + } + this.bytes = bytes; + this.offset = offset; + this.length = length; + this.type = bytes[offset + TAG_LENGTH_SIZE]; + } + + @Override + public ByteBuffer getBuffer() { + return ByteBuffer.wrap(bytes, offset, length); + } + + /** + * @return the tag type + */ + @Override + public byte getType() { + return this.type; + } + + /** + * @return Length of actual tag bytes within the backed buffer + */ + @Override + public int getTagLength() { + return this.length - INFRASTRUCTURE_SIZE; + } + + /** + * @return Offset of actual tag bytes within the backed buffer + */ + @Override + public int getTagOffset() { + return this.offset + INFRASTRUCTURE_SIZE; + } + + /** + * Returns tag value in a new byte array. + * Primarily for use client-side. If server-side, use + * {@link #getBuffer()} with appropriate {@link #getTagOffset()} and {@link #getTagLength()} + * instead to save on allocations. + * @return tag value in a new byte array. + */ + public byte[] getValue() { + int tagLength = getTagLength(); + byte[] tag = new byte[tagLength]; + Bytes.putBytes(tag, 0, bytes, getTagOffset(), tagLength); + return tag; + } + + /** + * Returns the total length of the entire tag entity + */ + @Override + public int getLength() { + return this.length; + } + + /** + * Returns the offset of the entire tag entity + */ + @Override + public int getOffset() { + return this.offset; + } + + @Override + public byte[] getBytes() { + return this.bytes; + } + + @Override + public boolean hasArray() { + return true; + } +} diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/BufferBackedTag.java hbase-common/src/main/java/org/apache/hadoop/hbase/BufferBackedTag.java new file mode 100644 index 0000000..469381a --- /dev/null +++ hbase-common/src/main/java/org/apache/hadoop/hbase/BufferBackedTag.java @@ -0,0 +1,122 @@ +/** + * Copyright The Apache Software Foundation + * + * 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; + +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.Bytes; + +public class BufferBackedTag implements Tag { + + private final byte type; + private final ByteBuffer buf; + private int offset = 0; + private int length = 0; + + public BufferBackedTag(byte tagType, String tag) { + this(tagType, ByteBuffer.wrap(Bytes.toBytes(tag))); + } + + /** + * @param tagType + * @param tag + */ + public BufferBackedTag(byte tagType, ByteBuffer tag) { + /** + * Format for a tag : taglength is serialized + * using 2 bytes only but as this will be unsigned, we can have max taglength of (Short.MAX_SIZE + * * 2) +1. It includes 1 byte type length and actual tag bytes length. + */ + int tagLength = tag.remaining() + TYPE_LENGTH_SIZE; + if (tagLength > MAX_TAG_LENGTH) { + throw new IllegalArgumentException( + "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH); + } + this.length = TAG_LENGTH_SIZE + tagLength; + this.buf = ByteBuffer.allocate(this.length); + int pos = ByteBufferUtils.putAsShort(buf, 0, tagLength); + pos = ByteBufferUtils.putByte(buf, pos, tagType); + ByteBufferUtils.copyFromBufferToBuffer(buf, tag, tag.position(), pos, tag.remaining()); + this.type = tagType; + } + + /** + * Creates a Tag from the specified byte array, starting at offset, and for length + * length. Presumes bytes content starting at offset is + * formatted as a Tag blob. + * + * @param buf + * @param offset + * offset to start of the Tag + * @param length + * length of the Tag + */ + public BufferBackedTag(ByteBuffer buf, int offset, int length) { + if (length > MAX_TAG_LENGTH) { + throw new IllegalArgumentException( + "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH); + } + this.buf = buf; + this.offset = offset; + this.length = length; + this.type = ByteBufferUtils.getByte(buf, offset + TAG_LENGTH_SIZE); + } + + @Override + public byte[] getBytes() { + throw new UnsupportedOperationException(); + } + + @Override + public ByteBuffer getBuffer() { + return this.buf; + } + + @Override + public boolean hasArray() { + return false; + } + + @Override + public byte getType() { + return this.type; + } + + @Override + public int getTagOffset() { + return this.offset + INFRASTRUCTURE_SIZE; + } + + @Override + public int getTagLength() { + return this.length - INFRASTRUCTURE_SIZE; + } + + @Override + public int getOffset() { + return this.offset; + } + + @Override + public int getLength() { + return this.length; + } +} diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferBackedKeyValue.java hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferBackedKeyValue.java new file mode 100644 index 0000000..f99732b --- /dev/null +++ hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferBackedKeyValue.java @@ -0,0 +1,667 @@ +/* + * 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; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.io.ByteBufferOutputStream; +import org.apache.hadoop.hbase.io.HeapSize; +import org.apache.hadoop.hbase.io.util.StreamUtils; +import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; +/** + * A ByteBuffer backed {@link Cell} representation. Used in the + * read path on the server side. + */ +public class ByteBufferBackedKeyValue implements Cell, SettableSequenceId, Cloneable, HeapSize, + OutputStreamWritable { + + public static final int ROW_KEY_OFFSET = KeyValue.ROW_OFFSET + KeyValue.ROW_LENGTH_SIZE; + + static final Log LOG = LogFactory.getLog(ByteBufferBackedKeyValue.class); + + public static final long HEAP_SIZE_STATIC = ClassSize.OBJECT /* the KeyValue object itself */ + + ClassSize.REFERENCE /* pointer to buffer */ + (2 * Bytes.SIZEOF_INT) /* offset & length */ + + Bytes.SIZEOF_LONG /* memstoreTS */ + Bytes.SIZEOF_SHORT /* rowLength cache */ + + (2 * Bytes.SIZEOF_INT) /* qualLength & keyLength cache */ + Bytes.SIZEOF_BYTE /*famLength*/; + + protected ByteBuffer buffer; + protected int offset; + protected int length; + + // cache these values + private short rowLength = -1; + private int qualLength = -1; + private int keyLength = -1; + private byte famLength = -1; + + public ByteBufferBackedKeyValue() { + + } + + public ByteBufferBackedKeyValue(ByteBuffer buffer, int offset, int length) { + this.buffer = buffer; + // The offset that we set here from the HFileReader should not have the + // arrayOffset. + // All the ByteBufferUtils API should use it internally. + this.offset = offset; + this.length = length; + } + + public ByteBufferBackedKeyValue(final ByteBuffer row, final ByteBuffer family, + final ByteBuffer qualifier, final long timestamp, Type type, final ByteBuffer value) { + this(row, 0, len(row), family, 0, len(family), qualifier, 0, len(qualifier), timestamp, type, + value, 0, len(value)); + } + + public ByteBufferBackedKeyValue(final ByteBuffer row, final ByteBuffer family, + final ByteBuffer qualifier, final long timestamp, Type type, final ByteBuffer value, + final List tags) { + this(row, 0, len(row), family, 0, len(family), qualifier, 0, len(qualifier), timestamp, type, + value, 0, len(value), tags); + } + + private static int len(ByteBuffer b) { + return b == null ? 0 : b.capacity(); + } + + public ByteBufferBackedKeyValue(final ByteBuffer row, final int roffset, final int rlength, + final ByteBuffer family, final int foffset, final int flength, final ByteBuffer qualifier, + final int qoffset, final int qlength, final long timestamp, final Type type, + final ByteBuffer value, final int voffset, final int vlength) { + this(row, roffset, rlength, family, foffset, flength, qualifier, qoffset, qlength, timestamp, + type, value, voffset, vlength, null); + } + + public ByteBufferBackedKeyValue(final ByteBuffer row, final int roffset, final int rlength, + final ByteBuffer family, final int foffset, final int flength, final ByteBuffer qualifier, + final int qoffset, final int qlength, final long timestamp, final Type type, + final ByteBuffer value, final int voffset, final int vlength, final List tags) { + this.buffer = createByteBuffer(row, roffset, rlength, family, foffset, flength, qualifier, + qoffset, qlength, timestamp, type, value, voffset, vlength, tags); + this.length = buffer.capacity(); + this.offset = 0; + } + + private static ByteBuffer createByteBuffer(final ByteBuffer row, final int roffset, + final int rlength, final ByteBuffer family, final int foffset, int flength, + final ByteBuffer qualifier, final int qoffset, int qlength, final long timestamp, + final Type type, final ByteBuffer value, final int voffset, int vlength, List tags) { + + checkParameters(row, rlength, family, flength, qlength, vlength); + + // Calculate length of tags area + int tagsLength = 0; + if (tags != null && !tags.isEmpty()) { + for (Tag t : tags) { + tagsLength += t.getLength(); + } + } + KeyValue.checkForTagsLength(tagsLength); + // Allocate right-sized byte array. + int keyLength = (int) KeyValue.getKeyDataStructureSize(rlength, flength, qlength); + ByteBuffer buffer = ByteBuffer.allocate((int) KeyValue.getKeyValueDataStructureSize(rlength, + flength, qlength, vlength, tagsLength)); + + // Write key, value and key row length. + int pos = 0; + pos = ByteBufferUtils.putInt(buffer, pos, keyLength); + + pos = ByteBufferUtils.putInt(buffer, pos, vlength); + pos = ByteBufferUtils.putShort(buffer, pos, (short) (rlength & 0x0000ffff)); + pos = ByteBufferUtils.copyFromBufferToBuffer(buffer, row, roffset, pos, rlength); + pos = ByteBufferUtils.putByte(buffer, pos, (byte) (flength & 0x0000ff)); + if (flength != 0) { + pos = ByteBufferUtils.copyFromBufferToBuffer(buffer, family, foffset, pos, flength); + } + if (qlength > 0) { + pos = ByteBufferUtils.copyFromBufferToBuffer(buffer, qualifier, qoffset, pos, qlength); + } + pos = ByteBufferUtils.putLong(buffer, pos, timestamp); + pos = ByteBufferUtils.putByte(buffer, pos, type.getCode()); + if (vlength > 0) { + pos = ByteBufferUtils.copyFromBufferToBuffer(buffer, value, voffset, pos, vlength); + } + // Add the tags after the value part + if (tagsLength > 0) { + pos = ByteBufferUtils.putShort(buffer, pos, tagsLength); + for (Tag t : tags) { + // This would be costly too. But generally this would not come in fake keys. For fake keys + // tags would not be needed at all + pos = ByteBufferUtils.copyFromBufferToBuffer(buffer, t.getBuffer(), t.getOffset(), pos, + t.getLength()); + } + } + return buffer; + } + + private static void checkParameters(final ByteBuffer row, final int rlength, + final ByteBuffer family, int flength, int qlength, int vlength) + throws IllegalArgumentException { + if (rlength > Short.MAX_VALUE) { + throw new IllegalArgumentException("Row > " + Short.MAX_VALUE); + } + if (row == null) { + throw new IllegalArgumentException("Row is null"); + } + if (flength > Byte.MAX_VALUE) { + throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE); + } + // Qualifier length + if (qlength > Integer.MAX_VALUE - rlength - flength) { + throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE); + } + // Key length + long longKeyLength = KeyValue.getKeyDataStructureSize(rlength, flength, qlength); + if (longKeyLength > Integer.MAX_VALUE) { + throw new IllegalArgumentException("keylength " + longKeyLength + " > " + Integer.MAX_VALUE); + } + // Value length + if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) { // FindBugs + // INT_VACUOUS_COMPARISON + throw new IllegalArgumentException("Value length " + vlength + " > " + + HConstants.MAXIMUM_VALUE_LENGTH); + } + } + + @Override + public byte[] getRowArray() { + throw new UnsupportedOperationException("byte[] not supported"); + } + + @Override + public int getRowOffset() { + return this.offset + ROW_KEY_OFFSET; + } + + @Override + public short getRowLength() { + if (rowLength == -1) { + rowLength = ByteBufferUtils.getShort(this.buffer, getKeyOffset()); + } + return rowLength; + } + + @Override + public byte[] getFamilyArray() { + throw new UnsupportedOperationException("byte[] not supported"); + } + + @Override + public int getFamilyOffset() { + if (rowLength == -1) { + getRowLength(); + } + return getFamilyOffset(rowLength); + } + + private int getFamilyOffset(int rlength) { + return this.offset + ROW_KEY_OFFSET + rlength + Bytes.SIZEOF_BYTE; + } + + @Override + public byte getFamilyLength() { + return getFamilyLength(getFamilyOffset()); + } + + private byte getFamilyLength(int foffset) { + if(famLength == -1) { + famLength = this.buffer.get(foffset - 1); + } + return famLength; + } + + @Override + public byte[] getQualifierArray() { + throw new UnsupportedOperationException("byte[] not supported"); + } + + @Override + public int getQualifierOffset() { + return getQualifierOffset(getFamilyOffset()); + } + + private int getQualifierOffset(int foffset) { + return foffset + getFamilyLength(foffset); + } + + @Override + public int getQualifierLength() { + if (qualLength == -1) { + if (rowLength == -1) { + getRowLength(); + } + qualLength = getQualifierLength(rowLength, getFamilyLength(this.offset + ROW_KEY_OFFSET + + rowLength + Bytes.SIZEOF_BYTE)); + } + return qualLength; + } + + private int getQualifierLength(int rlength, int flength) { + return getKeyLength() - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0); + } + + @Override + public long getTimestamp() { + int offset = getTimestampOffset(getKeyLength()); + return ByteBufferUtils.getLong(this.buffer, offset); + } + + /** + * @param keylength + * Pass if you have it to save on a int creation. + * @return Timestamp offset + */ + private int getTimestampOffset(final int keylength) { + return getKeyOffset() + keylength - KeyValue.TIMESTAMP_TYPE_SIZE; + } + + @Override + public byte getTypeByte() { + return ByteBufferUtils.getByte(this.buffer, this.offset + getKeyLength() - 1 + + KeyValue.ROW_OFFSET); + } + + @Override + @Deprecated + public long getMvccVersion() { + return getSequenceId(); + } + + @Override + public long getSequenceId() { + return seqId; + } + + private long seqId = 0; + + @Override + public byte[] getValueArray() { + throw new UnsupportedOperationException("byte[] not supported"); + } + + @Override + public int getValueOffset() { + int voffset = getKeyOffset() + getKeyLength(); + return voffset; + } + + @Override + public int getValueLength() { + int vlength = ByteBufferUtils.getInt(this.buffer, this.offset + Bytes.SIZEOF_INT); + return vlength; + } + + @Override + public byte[] getTagsArray() { + throw new UnsupportedOperationException("byte[] not supported"); + } + + @Override + public int getTagsOffset() { + int tagsLen = getTagsLength(); + if (tagsLen == 0) { + return this.offset + this.length; + } + return this.offset + this.length - tagsLen; + } + + @Override + public int getTagsLength() { + int tagsLen = this.length + - (getKeyLength() + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE); + if (tagsLen > 0) { + // There are some Tag bytes in the byte[]. So reduce 2 bytes which is + // added to denote the tags + // length + tagsLen -= KeyValue.TAGS_LENGTH_SIZE; + } + return tagsLen; + } + + @Override + public boolean hasArray() { + return false; + } + + @Override + @Deprecated + public byte[] getValue() { + byte[] val = new byte[getValueLength()]; + ByteBufferUtils.copyFromBufferToByteArray(val, getValueBuffer(), getValueOffset(), 0, getValueLength()); + return val; + } + + @Override + @Deprecated + public byte[] getFamily() { + byte[] fam = new byte[getFamilyLength()]; + ByteBufferUtils.copyFromBufferToByteArray(fam, getFamilyBuffer(), getFamilyOffset(), 0, getFamilyLength()); + return fam; + } + + @Override + @Deprecated + public byte[] getQualifier() { + byte[] qual = new byte[getQualifierLength()]; + ByteBufferUtils.copyFromBufferToByteArray(qual, getQualifierBuffer(), getQualifierOffset(), 0, getQualifierLength()); + return qual; + } + + @Override + @Deprecated + public byte[] getRow() { + // TODO : Avoid calling this method + byte[] row = new byte[getRowLength()]; + ByteBufferUtils.copyFromBufferToByteArray(row, getRowBuffer(), getRowOffset(), 0, getRowLength()); + return row; + } + + @Override + public void setSequenceId(long seqId) { + this.seqId = seqId; + } + + @Override + public long heapSize() { + long sum = HEAP_SIZE_STATIC; + sum += ClassSize.align((long)length); // number of bytes of data in the "bytes" + return ClassSize.align(sum); + } + + @Override + public ByteBuffer getRowBuffer() { + return buffer; + } + + @Override + public ByteBuffer getFamilyBuffer() { + return buffer; + } + + @Override + public ByteBuffer getQualifierBuffer() { + return buffer; + } + + @Override + public ByteBuffer getValueBuffer() { + return buffer; + } + + @Override + public ByteBuffer getTagsBuffer() { + return buffer; + } + + // TODO this is needed? + public int getKeyOffset() { + return this.offset + KeyValue.ROW_OFFSET; + } + + /** + * @return Length of key portion. + */ + public int getKeyLength() { + if (keyLength == -1) { + keyLength = ByteBufferUtils.getInt(this.buffer, this.offset); + } + return keyLength; + } + + public ByteBuffer getKey() { + // TODO : Need test cases for this + ByteBuffer dup = this.buffer.duplicate(); + dup.position(getKeyOffset()); + dup.limit(getKeyOffset() + getKeyLength()); + return dup.slice(); + } + + public ByteBuffer getBuffer() { + return this.buffer; + } + @Override + public boolean equals(Object other) { + if (!(other instanceof Cell)) { + return false; + } + return CellComparator.equals(this, (Cell)other); + } + + @Override + public int hashCode() { + return CellComparator.hashCode(this); + } + + @Override + public String toString() { + if (this.buffer == null || this.buffer.capacity() == 0) { + return "empty"; + } + return keyToString(this.buffer, this.offset + KeyValue.ROW_OFFSET, getKeyLength()) + "/vlen=" + + getValueLength() + "/seqid=" + seqId; + } + + public static String keyToString(final ByteBuffer b, final int o, final int l) { + if (b == null) + return ""; + int rowlength = ByteBufferUtils.getShort(b, o); + String row = ByteBufferUtils.toStringBinary(b, o + Bytes.SIZEOF_SHORT, rowlength); + int columnoffset = o + Bytes.SIZEOF_SHORT + 1 + rowlength; + int familylength = ByteBufferUtils.getByte(b, columnoffset-1); + int columnlength = l - ((columnoffset - o) + KeyValue.TIMESTAMP_TYPE_SIZE); + String family = familylength == 0 ? "" : ByteBufferUtils.toStringBinary(b, columnoffset, + familylength); + String qualifier = columnlength == 0 ? "" : ByteBufferUtils.toStringBinary(b, columnoffset + + familylength, columnlength - familylength); + long timestamp = ByteBufferUtils.getLong(b, o + (l - KeyValue.TIMESTAMP_TYPE_SIZE)); + String timestampStr = KeyValue.humanReadableTimestamp(timestamp); + byte type = b.get(o + l - 1); + return row + "/" + family + (family != null && family.length() > 0 ? ":" : "") + qualifier + + "/" + timestampStr + "/" + Type.codeToType(type); + } + + /** + * A simple form of {@link ByteBufferBackedKeyValue} that creates a + * ByteBufferBackedKeyValue with only the key part of the ByteBuffer. Mainly + * used in places where we need to compare two cells. Avoids copying of + * buffers in places like block index keys, we need to compare the key + * ByteBuffer with a cell. Hence create a ByteBufferBackedKeyOnlyKeyValue(aka + * Cell) that would help in comparing as two cells + */ + public static class ByteBufferBackedKeyOnlyKeyValue extends ByteBufferBackedKeyValue { + // cache these values + private short rowLength = -1; + private byte famLen = -1; + + public ByteBufferBackedKeyOnlyKeyValue() { + } + + public ByteBufferBackedKeyOnlyKeyValue(ByteBuffer b, int offset, int length) { + setKey(b, offset, length); + } + + @Override + public int getKeyOffset() { + return this.offset; + } + + /** + * A setter that helps to avoid object creation every time and whenever + * there is a need to create new KeyOnlyKeyValue. + * + * @param key + * @param offset + * @param length + */ + public void setKey(ByteBuffer key, int offset, int length) { + this.buffer = key; + this.offset = offset; + this.length = length; + // reset these two cached items + this.rowLength = -1; + this.famLen = -1; + } + + @Override + public int getRowOffset() { + return getKeyOffset() + Bytes.SIZEOF_SHORT; + } + + @Override + public byte getFamilyLength() { + if (famLen == -1) { + famLen = ByteBufferUtils.getByte(this.buffer, getFamilyOffset() - 1); + } + return famLen; + } + + @Override + public int getFamilyOffset() { + return this.offset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE; + } + + @Override + public int getQualifierLength() { + return getQualifierLength(getRowLength(), getFamilyLength()); + } + + @Override + public int getQualifierOffset() { + return getFamilyOffset() + getFamilyLength(); + } + + @Override + public int getKeyLength() { + return length; + } + + @Override + public short getRowLength() { + if (rowLength == -1) { + rowLength = ByteBufferUtils.getShort(this.buffer, getKeyOffset()); + } + return rowLength; + } + + @Override + public byte getTypeByte() { + return ByteBufferUtils.getByte(this.buffer, this.offset + getKeyLength() - 1); + } + + private int getQualifierLength(int rlength, int flength) { + return getKeyLength() - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0); + } + + @Override + public long getTimestamp() { + int tsOffset = getTimestampOffset(); + return ByteBufferUtils.getLong(this.buffer, tsOffset); + } + + public int getTimestampOffset() { + return getKeyOffset() + getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE; + } + + @Override + public ByteBuffer getTagsBuffer() { + return HConstants.EMPTY_BYTE_BUFFER; + } + + @Override + public int getTagsOffset() { + return 0; + } + + @Override + public byte[] getValueArray() { + throw new IllegalArgumentException("ByteBufferBackedKeyOnlyKeyValue don't have value."); + } + + @Override + public ByteBuffer getValueBuffer() { + throw new IllegalArgumentException("ByteBufferBackedKeyOnlyKeyValue don't have value."); + } + + @Override + public int getValueOffset() { + throw new IllegalArgumentException("ByteBufferBackedKeyOnlyKeyValue don't have value."); + } + + @Override + public int getValueLength() { + throw new IllegalArgumentException("ByteBufferBackedKeyOnlyKeyValue don't have value."); + } + + @Override + public int getTagsLength() { + return 0; + } + + @Override + public String toString() { + if (this.buffer == null || this.buffer.capacity() == 0) { + return "empty"; + } + return keyToString(this.buffer, this.offset, getKeyLength()) + "/vlen=0/mvcc=0"; + } + } + + public int getLength() { + return length; + } + + public int getOffset() { + return this.offset; + } + + @Override + public long oswrite(OutputStream out, boolean withTags) throws IOException { + // In KeyValueUtil#oswrite we do a Cell serialization as KeyValue. Any changes doing here, pls + // check KeyValueUtil#oswrite also and do necessary changes. + int lenToWrite = this.length; + if (!withTags) { + // TODO we can avoid this? + lenToWrite = this.getKeyLength() + this.getValueLength() + + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; + } + // This does same as DataOuput#writeInt (big-endian, etc.) + if (out instanceof ByteBufferOutputStream) { + ((ByteBufferOutputStream) out).writeInt(lenToWrite); + ((ByteBufferOutputStream) out).write(this.buffer, this.offset, lenToWrite); + } else { + StreamUtils.writeInt(out, lenToWrite); + if (this.buffer.hasArray()) { + out.write(this.buffer.array(), this.buffer.arrayOffset() + this.offset, lenToWrite); + } else { + // Here we have to create temp array and do copy!! + byte[] b = new byte[lenToWrite]; + ByteBufferUtils.copyFromBufferToByteArray(b, this.buffer, this.offset, 0, lenToWrite); + out.write(b); + } + } + return lenToWrite + Bytes.SIZEOF_INT; + } +} \ No newline at end of file diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java index 8f299cc..ba64b66 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java @@ -18,10 +18,11 @@ package org.apache.hadoop.hbase; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; - /** * The unit of storage in HBase consisting of the following fields:
*
@@ -235,4 +236,20 @@ public interface Cell {
    */
   @Deprecated
   byte[] getRow();
+
+  /**
+   * @return true if the Cell is backed by byte[]. In case of BB backed Cell we
+   *         return false.
+   */
+  boolean hasArray();
+  
+  ByteBuffer getRowBuffer();
+  
+  ByteBuffer getFamilyBuffer();
+  
+  ByteBuffer getQualifierBuffer();
+  
+  ByteBuffer getValueBuffer();
+  
+  ByteBuffer getTagsBuffer();
 }
\ No newline at end of file
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index cbb7ff3..78e17e0 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.hbase;
 
 import java.io.Serializable;
+import java.nio.ByteBuffer;
 import java.util.Comparator;
 
 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.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.primitives.Longs;
@@ -75,10 +77,18 @@ public class CellComparator implements Comparator, Serializable {
     }
   }
 
+  // These APIs would be used in DBEs only.  Which means they are not in the memstore. Hence
+  // we could only check for hasArray() or not. Combination would not be needed here
   public static int findCommonPrefixInRowPart(Cell left, Cell right, int rowCommonPrefix) {
-    return findCommonPrefix(left.getRowArray(), right.getRowArray(), left.getRowLength()
-        - rowCommonPrefix, right.getRowLength() - rowCommonPrefix, left.getRowOffset()
-        + rowCommonPrefix, right.getRowOffset() + rowCommonPrefix);
+    if (left.hasArray() && right.hasArray()) {
+      return findCommonPrefix(left.getRowArray(), right.getRowArray(), left.getRowLength()
+          - rowCommonPrefix, right.getRowLength() - rowCommonPrefix, left.getRowOffset()
+          + rowCommonPrefix, right.getRowOffset() + rowCommonPrefix);
+    } else {
+      return findCommonPrefix(left.getRowBuffer(), right.getRowBuffer(), left.getRowLength()
+          - rowCommonPrefix, right.getRowLength() - rowCommonPrefix, left.getRowOffset()
+          + rowCommonPrefix, right.getRowOffset() + rowCommonPrefix);
+    }
   }
 
   private static int findCommonPrefix(byte[] left, byte[] right, int leftLength, int rightLength,
@@ -91,19 +101,45 @@ public class CellComparator implements Comparator, Serializable {
     }
     return result;
   }
+  
+  private static int findCommonPrefix(ByteBuffer left, ByteBuffer right, int leftLength, int rightLength,
+      int leftOffset, int rightOffset) {
+    int length = Math.min(leftLength, rightLength);
+    int result = 0;
 
+    while (result < length && left.get(leftOffset + result) == right.get(rightOffset + result)) {
+      result++;
+    }
+    return result;
+  }
+
+  // Used in DBEs read path
   public static int findCommonPrefixInFamilyPart(Cell left, Cell right, int familyCommonPrefix) {
-    return findCommonPrefix(left.getFamilyArray(), right.getFamilyArray(), left.getFamilyLength()
-        - familyCommonPrefix, right.getFamilyLength() - familyCommonPrefix, left.getFamilyOffset()
-        + familyCommonPrefix, right.getFamilyOffset() + familyCommonPrefix);
+    if(left.hasArray() && right.hasArray()) {
+      return findCommonPrefix(left.getFamilyArray(), right.getFamilyArray(), left.getFamilyLength()
+          - familyCommonPrefix, right.getFamilyLength() - familyCommonPrefix, left.getFamilyOffset()
+          + familyCommonPrefix, right.getFamilyOffset() + familyCommonPrefix);
+    } else {
+      return findCommonPrefix(left.getFamilyBuffer(), right.getFamilyBuffer(),
+          left.getFamilyLength() - familyCommonPrefix,
+          right.getFamilyLength() - familyCommonPrefix,
+          left.getFamilyOffset() + familyCommonPrefix, right.getFamilyOffset() + familyCommonPrefix);
+    }
   }
 
   public static int findCommonPrefixInQualifierPart(Cell left, Cell right,
       int qualifierCommonPrefix) {
-    return findCommonPrefix(left.getQualifierArray(), right.getQualifierArray(),
-        left.getQualifierLength() - qualifierCommonPrefix, right.getQualifierLength()
-            - qualifierCommonPrefix, left.getQualifierOffset() + qualifierCommonPrefix,
-        right.getQualifierOffset() + qualifierCommonPrefix);
+    if(left.hasArray() && right.hasArray()) {
+      return findCommonPrefix(left.getQualifierArray(), right.getQualifierArray(),
+          left.getQualifierLength() - qualifierCommonPrefix, right.getQualifierLength()
+          - qualifierCommonPrefix, left.getQualifierOffset() + qualifierCommonPrefix,
+          right.getQualifierOffset() + qualifierCommonPrefix);
+    } else {
+      return findCommonPrefix(left.getQualifierBuffer(),
+          right.getQualifierBuffer(), left.getQualifierLength() - qualifierCommonPrefix,
+          right.getQualifierLength() - qualifierCommonPrefix, left.getQualifierOffset()
+              + qualifierCommonPrefix, right.getQualifierOffset() + qualifierCommonPrefix);
+    }
   }
 
   /**************** equals ****************************/
@@ -116,22 +152,32 @@ public class CellComparator implements Comparator, Serializable {
         && equalsType(a, b);
   }
 
-  public static boolean equalsRow(Cell a, Cell b){
-    return Bytes.equals(
-      a.getRowArray(), a.getRowOffset(), a.getRowLength(),
-      b.getRowArray(), b.getRowOffset(), b.getRowLength());
+  public static boolean equalsRow(Cell a, Cell b) {
+    if (a.hasArray() && b.hasArray()) {
+      return Bytes.equals(a.getRowArray(), a.getRowOffset(), a.getRowLength(), b.getRowArray(),
+          b.getRowOffset(), b.getRowLength());
+    }
+    return ByteBufferUtils.equals(a.getRowBuffer(), a.getRowOffset(), a.getRowLength(),
+        b.getRowBuffer(), b.getRowOffset(), b.getRowLength());
   }
 
-  public static boolean equalsFamily(Cell a, Cell b){
-    return Bytes.equals(
-      a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(),
-      b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength());
+  public static boolean equalsFamily(Cell a, Cell b) {
+    if (a.hasArray() && b.hasArray()) {
+      return Bytes.equals(a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(),
+          b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength());
+    }
+    return ByteBufferUtils.equals(a.getFamilyBuffer(), a.getFamilyOffset(), a.getFamilyLength(),
+        b.getFamilyBuffer(), b.getFamilyOffset(), b.getFamilyLength());
   }
 
-  public static boolean equalsQualifier(Cell a, Cell b){
-    return Bytes.equals(
-      a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(),
-      b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength());
+  public static boolean equalsQualifier(Cell a, Cell b) {
+    if (a.hasArray() && b.hasArray()) {
+      return Bytes.equals(a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(),
+          b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength());
+    }
+    return ByteBufferUtils.equals(a.getQualifierBuffer(), a.getQualifierOffset(),
+        a.getQualifierLength(), b.getQualifierBuffer(), b.getQualifierOffset(),
+        b.getQualifierLength());
   }
 
   public static boolean equalsTimestamp(Cell a, Cell b){
@@ -143,31 +189,65 @@ public class CellComparator implements Comparator, Serializable {
   }
 
   public static int compareColumns(final Cell left, final Cell right) {
-    int lfoffset = left.getFamilyOffset();
-    int rfoffset = right.getFamilyOffset();
-    int lclength = left.getQualifierLength();
-    int rclength = right.getQualifierLength();
-    int lfamilylength = left.getFamilyLength();
-    int rfamilylength = right.getFamilyLength();
-    int diff = compare(left.getFamilyArray(), lfoffset, lfamilylength, right.getFamilyArray(),
-        rfoffset, rfamilylength);
+    int diff = 0;
+    diff = compareFamilies(left, right);
     if (diff != 0) {
       return diff;
     } else {
-      return compare(left.getQualifierArray(), left.getQualifierOffset(), lclength,
-          right.getQualifierArray(), right.getQualifierOffset(), rclength);
+      return compareQualifiers(left, right);
     }
   }
 
   public static int compareFamilies(Cell left, Cell right) {
-    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
-        right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(),
+            left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
+            right.getFamilyLength());
+      } else {
+        return ByteBufferUtils.compareTo(left.getFamilyArray(), left.getFamilyOffset(),
+            left.getFamilyLength(), right.getFamilyBuffer(), right.getFamilyOffset(),
+            right.getFamilyLength());
+      }
+    } else {
+      if (right.hasArray()) {
+        int c = ByteBufferUtils.compareTo(right.getFamilyArray(), right.getFamilyOffset(),
+            right.getFamilyLength(), left.getFamilyBuffer(), left.getFamilyOffset(),
+            left.getFamilyLength());
+        return -c;
+      } else {
+        return ByteBufferUtils.compareTo(left.getFamilyBuffer(), left.getFamilyOffset(),
+            left.getFamilyLength(), right.getFamilyBuffer(), right.getFamilyOffset(),
+            right.getFamilyLength());
+      }
+    }
   }
 
   public static int compareQualifiers(Cell left, Cell right) {
-    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
-        left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
-        right.getQualifierLength());
+    return compareQualifiers(left, left.getQualifierOffset(), left.getQualifierLength(), right,
+        right.getQualifierOffset(), right.getQualifierLength());
+  }
+
+  public static int compareQualifiers(Cell left, int lOffset, int lLength, Cell right, int rOffset,
+      int rLength) {
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.compareTo(left.getQualifierArray(), lOffset, lLength,
+            right.getQualifierArray(), rOffset, rLength);
+      } else {
+        return ByteBufferUtils.compareTo(left.getQualifierArray(), lOffset, lLength,
+            right.getQualifierBuffer(), rOffset, rLength);
+      }
+    } else {
+      if (right.hasArray()) {
+        int c = ByteBufferUtils.compareTo(right.getQualifierArray(), rOffset, rLength,
+            left.getQualifierBuffer(), lOffset, lLength);
+        return -c;
+      } else {
+        return ByteBufferUtils.compareTo(left.getQualifierBuffer(), lOffset, lLength,
+            right.getQualifierBuffer(), rOffset, rLength);
+      }
+    }
   }
 
   public int compareFlatKey(Cell left, Cell right) {
@@ -183,8 +263,27 @@ public class CellComparator implements Comparator, Serializable {
    * so can't be treated as plain byte arrays as this method does.
    */
   public static int compareRows(final Cell left, final Cell right) {
-    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+    return compareRows(left, left.getRowOffset(), left.getRowLength(), right, right.getRowOffset(),
+        right.getRowLength());
+  }
+
+  public static int compareRows(Cell left, int lrOffset, int lrLength, Cell right, int rrOffset,
+      int rrLength) {
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.compareTo(left.getRowArray(), lrOffset, lrLength, right.getRowArray(),
+            rrOffset, rrLength);
+      }
+      return ByteBufferUtils.compareTo(left.getRowArray(), lrOffset, lrLength,
+          right.getRowBuffer(), rrOffset, rrLength);
+    } else {
+      if (right.hasArray()) {
+        return ByteBufferUtils.compareTo(left.getRowBuffer(), lrOffset, lrLength,
+            right.getRowArray(), rrOffset, rrLength);
+      }
+      return ByteBufferUtils.compareTo(left.getRowBuffer(), lrOffset, lrLength,
+          right.getRowBuffer(), rrOffset, rrLength);
+    }
   }
 
   /**
@@ -216,10 +315,7 @@ public class CellComparator implements Comparator, Serializable {
     boolean sameFamilySize = (leftCell.getFamilyLength() == rightCell.getFamilyLength());
     if (!sameFamilySize) {
       // comparing column family is enough.
-
-      return Bytes.compareTo(leftCell.getFamilyArray(), leftCell.getFamilyOffset(),
-          leftCell.getFamilyLength(), rightCell.getFamilyArray(), rightCell.getFamilyOffset(),
-          rightCell.getFamilyLength());
+      return compareFamilies(leftCell, rightCell);
     }
     int diff = compareColumns(leftCell, rightCell);
     if (diff != 0) return diff;
@@ -275,13 +371,15 @@ public class CellComparator implements Comparator, Serializable {
 
   private static int calculateHashForKeyValue(Cell cell) {
     //pre-calculate the 3 hashes made of byte ranges
-    int rowHash = Bytes.hashCode(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
-    int familyHash =
-      Bytes.hashCode(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
-    int qualifierHash = Bytes.hashCode(cell.getQualifierArray(), cell.getQualifierOffset(),
-      cell.getQualifierLength());
-
-    //combine the 6 sub-hashes
+    int rowHash = 0;
+    int familyHash = 0;
+    int qualifierHash = 0;
+    rowHash = ByteBufferUtils.hashCode(cell.getRowBuffer(), cell.getRowOffset(),
+        cell.getRowLength());
+    familyHash = ByteBufferUtils.hashCode(cell.getFamilyBuffer(), cell.getFamilyOffset(),
+        cell.getFamilyLength());
+    qualifierHash = ByteBufferUtils.hashCode(cell.getQualifierBuffer(), cell.getQualifierOffset(),
+        cell.getQualifierLength());
     int hash = 31 * rowHash + familyHash;
     hash = 31 * hash + qualifierHash;
     hash = 31 * hash + (int)cell.getTimestamp();
@@ -289,7 +387,6 @@ public class CellComparator implements Comparator, Serializable {
     return hash;
   }
 
-
   /******************** lengths *************************/
 
   public static boolean areKeyLengthsEqual(Cell a, Cell b) {
@@ -310,26 +407,91 @@ public class CellComparator implements Comparator, Serializable {
     return Bytes.compareTo(left, leftOffset, leftLength, right, rightOffset, rightLength);
   }
 
+  // Used in DBEs
   public static int compareCommonRowPrefix(Cell left, Cell right, int rowCommonPrefix) {
-    return compare(left.getRowArray(), left.getRowOffset() + rowCommonPrefix, left.getRowLength()
-        - rowCommonPrefix, right.getRowArray(), right.getRowOffset() + rowCommonPrefix,
-        right.getRowLength() - rowCommonPrefix);
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.compareTo(left.getRowArray(), left.getRowOffset() + rowCommonPrefix,
+            left.getRowLength() - rowCommonPrefix, right.getRowArray(), right.getRowOffset()
+                + rowCommonPrefix, right.getRowLength() - rowCommonPrefix);
+      } else {
+        return ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset() + rowCommonPrefix,
+            left.getRowLength() - rowCommonPrefix, right.getRowBuffer(), right.getRowOffset()
+                + rowCommonPrefix, right.getRowLength() - rowCommonPrefix);
+      }
+    } else {
+      if (right.hasArray()) {
+        int c = ByteBufferUtils.compareTo(right.getRowArray(), right.getRowOffset()
+            + rowCommonPrefix, right.getRowLength() - rowCommonPrefix, left.getRowBuffer(),
+            left.getRowOffset() + rowCommonPrefix, left.getRowLength() - rowCommonPrefix);
+        return -c;
+      } else {
+        return ByteBufferUtils.compareTo(left.getRowBuffer(),
+            left.getRowOffset() + rowCommonPrefix, left.getRowLength() - rowCommonPrefix,
+            right.getRowBuffer(), right.getRowOffset() + rowCommonPrefix, right.getRowLength()
+                - rowCommonPrefix);
+      }
+    }
   }
 
   public static int compareCommonFamilyPrefix(Cell left, Cell right,
       int familyCommonPrefix) {
-    return compare(left.getFamilyArray(), left.getFamilyOffset() + familyCommonPrefix,
-        left.getFamilyLength() - familyCommonPrefix, right.getFamilyArray(),
-        right.getFamilyOffset() + familyCommonPrefix,
-        right.getFamilyLength() - familyCommonPrefix);
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset() + familyCommonPrefix,
+            left.getFamilyLength() - familyCommonPrefix, right.getFamilyArray(),
+            right.getFamilyOffset() + familyCommonPrefix, right.getFamilyLength()
+                - familyCommonPrefix);
+      } else {
+        return ByteBufferUtils.compareTo(left.getFamilyArray(), left.getFamilyOffset()
+            + familyCommonPrefix, left.getFamilyLength() - familyCommonPrefix,
+            right.getFamilyBuffer(), right.getFamilyOffset() + familyCommonPrefix,
+            right.getFamilyLength() - familyCommonPrefix);
+      }
+    } else {
+      if (right.hasArray()) {
+        int c = ByteBufferUtils.compareTo(right.getFamilyArray(), right.getFamilyOffset()
+            + familyCommonPrefix, right.getFamilyLength() - familyCommonPrefix,
+            left.getFamilyBuffer(), left.getFamilyOffset() + familyCommonPrefix,
+            left.getFamilyLength() - familyCommonPrefix);
+        return -c;
+      } else {
+        return ByteBufferUtils.compareTo(left.getFamilyBuffer(), left.getFamilyOffset()
+            + familyCommonPrefix, left.getFamilyLength() - familyCommonPrefix,
+            right.getFamilyBuffer(), right.getFamilyOffset() + familyCommonPrefix,
+            right.getFamilyLength() - familyCommonPrefix);
+      }
+    }
   }
 
   public static int compareCommonQualifierPrefix(Cell left, Cell right,
       int qualCommonPrefix) {
-    return compare(left.getQualifierArray(), left.getQualifierOffset() + qualCommonPrefix,
-        left.getQualifierLength() - qualCommonPrefix, right.getQualifierArray(),
-        right.getQualifierOffset() + qualCommonPrefix, right.getQualifierLength()
-            - qualCommonPrefix);
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset()
+            + qualCommonPrefix, left.getQualifierLength() - qualCommonPrefix,
+            right.getQualifierArray(), right.getQualifierOffset() + qualCommonPrefix,
+            right.getQualifierLength() - qualCommonPrefix);
+      } else {
+        return ByteBufferUtils.compareTo(left.getQualifierArray(), left.getQualifierOffset()
+            + qualCommonPrefix, left.getQualifierLength() - qualCommonPrefix,
+            right.getQualifierBuffer(), right.getQualifierOffset() + qualCommonPrefix,
+            right.getQualifierLength() - qualCommonPrefix);
+      }
+    } else {
+      if (right.hasArray()) {
+        int c = ByteBufferUtils.compareTo(right.getQualifierArray(), right.getQualifierOffset()
+            + qualCommonPrefix, right.getQualifierLength() - qualCommonPrefix,
+            left.getQualifierBuffer(), left.getQualifierOffset() + qualCommonPrefix,
+            left.getQualifierLength() - qualCommonPrefix);
+        return -c;
+      } else {
+        return ByteBufferUtils.compareTo(left.getQualifierBuffer(), left.getQualifierOffset()
+            + qualCommonPrefix, left.getQualifierLength() - qualCommonPrefix,
+            right.getQualifierBuffer(), right.getQualifierOffset() + qualCommonPrefix,
+            right.getQualifierLength() - qualCommonPrefix);
+      }
+    }
   }
 
   /***************** special cases ****************************/
@@ -412,11 +574,13 @@ public class CellComparator implements Comparator, Serializable {
       throw new IllegalArgumentException("Left row sorts after right row; left=" +
         CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
     }
+    byte[] midRow = null;
     if (diff < 0) {
       // Left row is < right row.
-      byte [] midRow = getMinimumMidpointArray(left.getRowArray(), left.getRowOffset(),
-          left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+        midRow = getMinimumMidpointArray(left.getRowBuffer(), left.getRowOffset(),
+            left.getRowLength(),
+            right.getRowBuffer(), right.getRowOffset(), right.getRowLength());
+      //}
       // If midRow is null, just return 'right'.  Can't do optimization.
       if (midRow == null) return right;
       return CellUtil.createCell(midRow);
@@ -428,15 +592,20 @@ public class CellComparator implements Comparator, Serializable {
           CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
     }
     if (diff < 0) {
-      byte [] midRow = getMinimumMidpointArray(left.getFamilyArray(), left.getFamilyOffset(),
-          left.getFamilyLength(),
-        right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
-      // If midRow is null, just return 'right'.  Can't do optimization.
+        midRow = getMinimumMidpointArray(left.getFamilyBuffer(), left.getFamilyOffset(),
+            left.getFamilyLength(),
+            right.getFamilyBuffer(), right.getFamilyOffset(), right.getFamilyLength());
       if (midRow == null) return right;
-      // Return new Cell where we use right row and then a mid sort family.
-      return CellUtil.createCell(right.getRowArray(), right.getRowOffset(), right.getRowLength(),
-        midRow, 0, midRow.length, HConstants.EMPTY_BYTE_ARRAY, 0,
-        HConstants.EMPTY_BYTE_ARRAY.length);
+      if (right.hasArray()) {
+        // Return new Cell where we use right row and then a mid sort family.
+        return CellUtil.createCell(right.getRowArray(), right.getRowOffset(), right.getRowLength(),
+            midRow, 0, midRow.length, HConstants.EMPTY_BYTE_ARRAY, 0,
+            HConstants.EMPTY_BYTE_ARRAY.length);
+      } else {
+        return new FirstOnColumnCell(right.getRowBuffer(), right.getRowOffset(),
+            right.getRowLength(), ByteBuffer.wrap(midRow), 0, (byte) midRow.length,
+            HConstants.EMPTY_BYTE_BUFFER, 0, 0);
+      }
     }
     // Families are same. Compare on qualifiers.
     diff = compareQualifiers(left, right);
@@ -445,54 +614,58 @@ public class CellComparator implements Comparator, Serializable {
           CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
     }
     if (diff < 0) {
-      byte [] midRow = getMinimumMidpointArray(left.getQualifierArray(), left.getQualifierOffset(),
-          left.getQualifierLength(),
-        right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
-      // If midRow is null, just return 'right'.  Can't do optimization.
+      midRow = getMinimumMidpointArray(left.getQualifierBuffer(), left.getQualifierOffset(),
+          left.getQualifierLength(), right.getQualifierBuffer(), right.getQualifierOffset(),
+          right.getQualifierLength());
+      // If midRow is null, just return 'right'. Can't do optimization.
       if (midRow == null) return right;
-      // Return new Cell where we use right row and family and then a mid sort qualifier.
-      return CellUtil.createCell(right.getRowArray(), right.getRowOffset(), right.getRowLength(),
-        right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength(),
-        midRow, 0, midRow.length);
+      if (right.hasArray()) {
+        return CellUtil.createCell(right.getRowArray(), right.getRowOffset(), right.getRowLength(),
+            right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength(), midRow, 0,
+            midRow.length);
+      } else {
+        return new FirstOnColumnCell(right.getRowBuffer(), right.getRowOffset(),
+            right.getRowLength(), right.getFamilyBuffer(), right.getFamilyOffset(),
+            right.getFamilyLength(), ByteBuffer.wrap(midRow), 0, midRow.length);
+      }
     }
     // No opportunity for optimization. Just return right key.
     return right;
   }
 
   /**
-   * @param leftArray
+   * @param left
    * @param leftOffset
    * @param leftLength
-   * @param rightArray
+   * @param right
    * @param rightOffset
    * @param rightLength
    * @return Return a new array that is between left and right and minimally sized else just return
    * null as indicator that we could not create a mid point.
    */
-  private static byte [] getMinimumMidpointArray(final byte [] leftArray, final int leftOffset,
-        final int leftLength,
-      final byte [] rightArray, final int rightOffset, final int rightLength) {
+  private static byte[] getMinimumMidpointArray(final ByteBuffer left, final int leftOffset,
+      final int leftLength, final ByteBuffer right, final int rightOffset, final int rightLength) {
     // rows are different
     int minLength = leftLength < rightLength ? leftLength : rightLength;
     short diffIdx = 0;
     while (diffIdx < minLength &&
-        leftArray[leftOffset + diffIdx] == rightArray[rightOffset + diffIdx]) {
+        left.get(leftOffset + diffIdx) == right.get(rightOffset + diffIdx)) {
       diffIdx++;
     }
     byte [] minimumMidpointArray = null;
     if (diffIdx >= minLength) {
       // leftKey's row is prefix of rightKey's.
       minimumMidpointArray = new byte[diffIdx + 1];
-      System.arraycopy(rightArray, rightOffset, minimumMidpointArray, 0, diffIdx + 1);
+      ByteBufferUtils.copyFromBufferToByteArray(minimumMidpointArray, right, rightOffset, 0, diffIdx + 1);
     } else {
-      int diffByte = leftArray[leftOffset + diffIdx];
-      if ((0xff & diffByte) < 0xff && (diffByte + 1) < (rightArray[rightOffset + diffIdx] & 0xff)) {
+      int diffByte = left.get(leftOffset + diffIdx);
+      if ((0xff & diffByte) < 0xff && (diffByte + 1) < (right.get(rightOffset + diffIdx) & 0xff)) {
         minimumMidpointArray = new byte[diffIdx + 1];
-        System.arraycopy(leftArray, leftOffset, minimumMidpointArray, 0, diffIdx);
+        ByteBufferUtils.copyFromBufferToByteArray(minimumMidpointArray, left, leftOffset, 0, diffIdx);
         minimumMidpointArray[diffIdx] = (byte) (diffByte + 1);
       } else {
         minimumMidpointArray = new byte[diffIdx + 1];
-        System.arraycopy(rightArray, rightOffset, minimumMidpointArray, 0, diffIdx + 1);
+        ByteBufferUtils.copyFromBufferToByteArray(minimumMidpointArray, right, rightOffset, 0, diffIdx + 1);
       }
     }
     return minimumMidpointArray;
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 7b68eee..72d28b8 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
 
+
 /**
  * Utility methods helpful slinging {@link Cell} instances.
  */
@@ -96,9 +97,7 @@ public final class CellUtil {
   }
 
   /**
-   * Returns tag value in a new byte array. If server-side, use
-   * {@link Tag#getBuffer()} with appropriate {@link Tag#getTagOffset()} and
-   * {@link Tag#getTagLength()} instead to save on allocations.
+   * Returns tag value in a new byte array.
    * @param cell
    * @return tag value in a new byte array.
    */
@@ -112,26 +111,46 @@ public final class CellUtil {
   /******************** copyTo **********************************/
 
   public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
+    if(cell.hasArray()) {
     System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
       cell.getRowLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToByteArray(destination, cell.getRowBuffer(), 
+          cell.getRowOffset(), destinationOffset, cell.getRowLength());
+    }
     return destinationOffset + cell.getRowLength();
   }
 
   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset,
-      cell.getFamilyLength());
+    if (cell.hasArray()) {
+      System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
+          destinationOffset, cell.getFamilyLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToByteArray(destination, cell.getFamilyBuffer(), 
+          cell.getFamilyOffset(), destinationOffset, cell.getFamilyLength());
+    }
     return destinationOffset + cell.getFamilyLength();
   }
 
   public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
-      destinationOffset, cell.getQualifierLength());
+    if (cell.hasArray()) {
+      System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
+          destinationOffset, cell.getQualifierLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToByteArray(destination, cell.getQualifierBuffer(), 
+          cell.getQualifierOffset(), destinationOffset, cell.getQualifierLength());
+    }
     return destinationOffset + cell.getQualifierLength();
   }
 
   public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
-        cell.getValueLength());
+    if (cell.hasArray()) {
+      System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
+          cell.getValueLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToByteArray(destination, cell.getValueBuffer(), 
+          cell.getValueOffset(), destinationOffset, cell.getValueLength());
+    }
     return destinationOffset + cell.getValueLength();
   }
 
@@ -143,27 +162,31 @@ public final class CellUtil {
    * @return position after tags
    */
   public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
-        cell.getTagsLength());
+    if (cell.hasArray()) {
+      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
+          cell.getTagsLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToByteArray(destination, cell.getTagsBuffer(),
+          cell.getTagsOffset(), destinationOffset, cell.getTagsLength());
+    }
     return destinationOffset + cell.getTagsLength();
   }
 
   /********************* misc *************************************/
 
   public static byte getRowByte(Cell cell, int index) {
-    return cell.getRowArray()[cell.getRowOffset() + index];
+    if (cell.hasArray()) {
+      return cell.getRowArray()[cell.getRowOffset() + index];
+    }
+    return ByteBufferUtils.getByte(cell.getRowBuffer(), cell.getRowOffset() + index);
   }
 
   public static ByteBuffer getValueBufferShallowCopy(Cell cell) {
-    ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(),
-      cell.getValueLength());
-    return buffer;
+    return cell.getValueBuffer();
   }
 
   public static ByteBuffer getQualifierBufferShallowCopy(Cell cell) {
-    ByteBuffer buffer = ByteBuffer.wrap(cell.getQualifierArray(), cell.getQualifierOffset(),
-        cell.getQualifierLength());
-    return buffer;
+    return cell.getQualifierBuffer();
   }
 
   public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier,
@@ -282,18 +305,8 @@ public final class CellUtil {
    */
   public static CellScanner createCellScanner(final Iterable cellIterable) {
     if (cellIterable == null) return null;
-    return createCellScanner(cellIterable.iterator());
-  }
-
-  /**
-   * @param cells
-   * @return CellScanner interface over cellIterable or null if cells is
-   * null
-   */
-  public static CellScanner createCellScanner(final Iterator cells) {
-    if (cells == null) return null;
     return new CellScanner() {
-      private final Iterator iterator = cells;
+      private Iterator iterator = cellIterable.iterator();
       private Cell current = null;
 
       @Override
@@ -304,7 +317,7 @@ public final class CellUtil {
       @Override
       public boolean advance() {
         boolean hasNext = this.iterator.hasNext();
-        this.current = hasNext? this.iterator.next(): null;
+        this.current = hasNext ? this.iterator.next() : null;
         return hasNext;
       }
     };
@@ -374,49 +387,110 @@ public final class CellUtil {
    * @return True if the rows in left and right Cells match
    */
   public static boolean matchingRow(final Cell left, final Cell right) {
-    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+    return matchingRow(left, left.getRowOffset(), left.getRowLength(), right, right.getRowOffset(),
+        right.getRowLength());
+  }
+
+  public static boolean matchingRow(Cell left, int lrOffset, int lrLength, Cell right,
+      int rrOffset, int rrLength) {
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.equals(left.getRowArray(), lrOffset, lrLength, right.getRowArray(), rrOffset,
+            rrLength);
+      }
+      return ByteBufferUtils.equals(left.getRowArray(), lrOffset, lrLength, right.getRowBuffer(),
+          rrOffset, rrLength);
+    } else {
+      if (right.hasArray()) {
+        return ByteBufferUtils.equals(left.getRowBuffer(), lrOffset, lrLength, right.getRowArray(),
+            rrOffset, rrLength);
+      }
+      return ByteBufferUtils.equals(left.getRowBuffer(), lrOffset, lrLength, right.getRowBuffer(),
+          rrOffset, rrLength);
+    }
   }
 
   public static boolean matchingRow(final Cell left, final byte[] buf) {
-    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, 0,
-        buf.length);
+    if (left.hasArray()) {
+      return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, 0,
+          buf.length);
+    }
+    return ByteBufferUtils.equals(left.getRowBuffer(), left.getRowOffset(), left.getRowLength(),
+        buf, 0, buf.length);
   }
 
-  public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
+  public static boolean matchingRow(final Cell left, final ByteBuffer buf, final int offset,
       final int length) {
-    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
-        length);
+    if (left.hasArray()) {
+      return ByteBufferUtils.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
+          buf, offset, length);
+    }
+    return ByteBufferUtils.equals(left.getRowBuffer(), left.getRowOffset(), left.getRowLength(),
+        buf, offset, length);
   }
 
   public static boolean matchingFamily(final Cell left, final Cell right) {
-    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
-        right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+            right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
+      }
+      return ByteBufferUtils.equals(left.getFamilyArray(), left.getFamilyOffset(),
+          left.getFamilyLength(), right.getFamilyBuffer(), right.getFamilyOffset(),
+          right.getFamilyLength());
+    } else {
+      if (right.hasArray()) {
+        return ByteBufferUtils.equals(left.getFamilyBuffer(), left.getFamilyOffset(),
+            left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
+            right.getFamilyLength());
+      }
+      return ByteBufferUtils.equals(left.getFamilyBuffer(), left.getFamilyOffset(),
+          left.getFamilyLength(), right.getFamilyBuffer(), right.getFamilyOffset(),
+          right.getFamilyLength());
+    }
   }
 
   public static boolean matchingFamily(final Cell left, final byte[] buf) {
-    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
-        0, buf.length);
+    return matchingFamily(left, buf, 0, buf.length);
   }
 
   public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
       final int length) {
-    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
-        offset, length);
+    if (left.hasArray()) {
+      return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+          buf, offset, length);
+    }
+    return ByteBufferUtils.equals(left.getFamilyBuffer(), left.getFamilyOffset(),
+        left.getFamilyLength(), buf, offset, length);
   }
 
   public static boolean matchingQualifier(final Cell left, final Cell right) {
-    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
-        left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
-        right.getQualifierLength());
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
+            left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
+            right.getQualifierLength());
+      }
+      return ByteBufferUtils.equals(left.getQualifierArray(), left.getQualifierOffset(),
+          left.getQualifierLength(), right.getQualifierBuffer(), right.getQualifierOffset(),
+          right.getQualifierLength());
+    } else {
+      if (right.hasArray()) {
+        return ByteBufferUtils.equals(left.getQualifierBuffer(), left.getQualifierOffset(),
+            left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
+            right.getQualifierLength());
+      }
+      return ByteBufferUtils.equals(left.getQualifierBuffer(), left.getQualifierOffset(),
+          left.getQualifierLength(), right.getQualifierBuffer(), right.getQualifierOffset(),
+          right.getQualifierLength());
+    }
   }
 
   public static boolean matchingQualifier(final Cell left, final byte[] buf) {
     if (buf == null) {
       return left.getQualifierLength() == 0;
     }
-    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
-        left.getQualifierLength(), buf, 0, buf.length);
+    return matchingQualifier(left, buf, 0, buf.length);
   }
 
   public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
@@ -424,37 +498,53 @@ public final class CellUtil {
     if (buf == null) {
       return left.getQualifierLength() == 0;
     }
-    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
+    if (left.hasArray()) {
+      return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
+          left.getQualifierLength(), buf, offset, length);
+    }
+    return ByteBufferUtils.equals(left.getQualifierBuffer(), left.getQualifierOffset(),
         left.getQualifierLength(), buf, offset, length);
   }
 
   public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
-    if (!matchingFamily(left, fam))
-      return false;
+    if (!matchingFamily(left, fam)) return false;
     return matchingQualifier(left, qual);
   }
 
   public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
       final int flength, final byte[] qual, final int qoffset, final int qlength) {
-    if (!matchingFamily(left, fam, foffset, flength))
-      return false;
+    if (!matchingFamily(left, fam, foffset, flength)) return false;
     return matchingQualifier(left, qual, qoffset, qlength);
   }
 
   public static boolean matchingColumn(final Cell left, final Cell right) {
-    if (!matchingFamily(left, right))
-      return false;
+    if (!matchingFamily(left, right)) return false;
     return matchingQualifier(left, right);
   }
 
   public static boolean matchingValue(final Cell left, final Cell right) {
-    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
-        right.getValueArray(), right.getValueOffset(), right.getValueLength());
+    if (left.hasArray()) {
+      if (right.hasArray()) {
+        return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
+            right.getValueArray(), right.getValueOffset(), right.getValueLength());
+      }
+      return ByteBufferUtils.equals(left.getValueArray(), left.getValueOffset(),
+          left.getValueLength(), right.getValueBuffer(), right.getValueOffset(),
+          right.getValueLength());
+    }
+    if (right.hasArray()) {
+      return ByteBufferUtils.equals(left.getValueBuffer(), left.getValueOffset(),
+          left.getValueLength(), right.getValueArray(), right.getValueOffset(),
+          right.getValueLength());
+    }
+    return ByteBufferUtils.equals(left.getValueBuffer(), left.getValueOffset(),
+        left.getValueLength(), right.getValueBuffer(), right.getValueOffset(),
+        right.getValueLength());
   }
 
   public static boolean matchingValue(final Cell left, final byte[] buf) {
-    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
-        buf.length);
+    return ByteBufferUtils.equals(left.getValueBuffer(), left.getValueOffset(),
+        left.getValueLength(), buf, 0, buf.length);
   }
 
   /**
@@ -514,8 +604,10 @@ public final class CellUtil {
    */
   public static int estimatedSerializedSizeOf(final Cell cell) {
     // If a KeyValue, we can give a good estimate of size.
-    if (cell instanceof KeyValue) {
-      return ((KeyValue)cell).getLength() + Bytes.SIZEOF_INT;
+    if (cell instanceof ByteBufferBackedKeyValue) {
+      return ((ByteBufferBackedKeyValue) cell).getLength() + Bytes.SIZEOF_INT;
+    } else if (cell instanceof KeyValue) {
+      return ((KeyValue) cell).getLength() + Bytes.SIZEOF_INT;
     }
     // TODO: Should we add to Cell a sizeOf?  Would it help? Does it make sense if Cell is
     // prefix encoded or compressed?
@@ -576,7 +668,9 @@ public final class CellUtil {
    * @param offset
    * @param length
    * @return iterator for the tags
+   * @deprecated Use {@link #tagsIterator(Cell)} instead
    */
+  @Deprecated
   public static Iterator tagsIterator(final byte[] tags, final int offset, final int length) {
     return new Iterator() {
       private int pos = offset;
@@ -591,7 +685,48 @@ public final class CellUtil {
       public Tag next() {
         if (hasNext()) {
           int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
-          Tag tag = new Tag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          Tag tag = new ArrayBackedTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
+          return tag;
+        }
+        return null;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  /********************* tags *************************************/
+  /**
+   * Util method to iterate through the tags
+   * @param cell
+   * @return iterator for the tags
+   */
+  public static Iterator tagsIterator(final Cell cell) {
+    return new Iterator() {
+      private int pos = cell.getTagsOffset();
+      private int endOffset = pos + cell.getTagsLength() - 1;
+
+      @Override
+      public boolean hasNext() {
+        return this.pos < endOffset;
+      }
+
+      @Override
+      public Tag next() {
+        if (hasNext()) {
+          int curTagLen;
+          Tag tag;
+          if (cell.hasArray()) {
+            curTagLen = Bytes.readAsInt(cell.getTagsArray(), pos, Tag.TAG_LENGTH_SIZE);
+            tag = new ArrayBackedTag(cell.getTagsArray(), pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          } else {
+            curTagLen = ByteBufferUtils.getAsInt(cell.getTagsBuffer(), pos, Tag.TAG_LENGTH_SIZE);
+            tag = new BufferBackedTag(cell.getTagsBuffer(), pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          }
           this.pos += Bytes.SIZEOF_SHORT + curTagLen;
           return tag;
         }
@@ -707,15 +842,43 @@ public final class CellUtil {
   public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException {
     short rowLen = cell.getRowLength();
     out.writeShort(rowLen);
-    out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
+    if (cell.hasArray()) {
+      out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
+    } else {
+      ByteBufferUtils.copyBufferToStream(out, cell.getRowBuffer(), cell.getRowOffset(), rowLen);
+    }
     byte fLen = cell.getFamilyLength();
     out.writeByte(fLen);
-    out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
-    out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
+    if (cell.hasArray()) {
+      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen);
+      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
+    } else {
+      ByteBufferUtils.copyBufferToStream(out, cell.getFamilyBuffer(), cell.getFamilyOffset(), fLen);
+      ByteBufferUtils.copyBufferToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset(),
+          cell.getQualifierLength());
+    }
     out.writeLong(cell.getTimestamp());
     out.writeByte(cell.getTypeByte());
   }
 
+  public static void writeValue(DataOutputStream out, Cell cell, int vlength) throws IOException {
+    if (cell.hasArray()) {
+      out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
+    } else {
+      ByteBufferUtils
+          .copyBufferToStream(out, cell.getValueBuffer(), cell.getValueOffset(), vlength);
+    }
+  }
+
+  public static void writeTags(DataOutputStream out, Cell cell, int tagsLength) throws IOException {
+    if (cell.hasArray()) {
+      out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
+    } else {
+      ByteBufferUtils.copyBufferToStream(out, cell.getTagsBuffer(), cell.getTagsOffset(),
+          tagsLength);
+    }
+  }
+
   /**
    * @param cell
    * @return The Key portion of the passed cell as a String.
@@ -776,7 +939,12 @@ public final class CellUtil {
       commonPrefix -= KeyValue.ROW_LENGTH_SIZE;
     }
     if (rLen > commonPrefix) {
-      out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rLen - commonPrefix);
+      if (cell.hasArray()) {
+        out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rLen - commonPrefix);
+      } else {
+        ByteBufferUtils.copyBufferToStream(out, cell.getRowBuffer(), cell.getRowOffset()
+            + commonPrefix, rLen - commonPrefix);
+      }
     }
   }
 
@@ -807,8 +975,8 @@ public final class CellUtil {
           Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
     }
     // Compare the RKs
-    int rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(),
-        rLen1, c2.getRowArray(), c2.getRowOffset(), rLen2);
+    int rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowBuffer(), c1.getRowOffset(),
+        rLen1, c2.getRowBuffer(), c2.getRowOffset(), rLen2);
     commonPrefix += rkCommonPrefix;
     if (rkCommonPrefix != rLen1) {
       // Early out when RK is not fully matching.
@@ -829,8 +997,8 @@ public final class CellUtil {
       // CF lengths are same so there is one more byte common in key part
       commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
       // Compare the CF names
-      int fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(),
-          c1.getFamilyOffset(), fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
+      int fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyBuffer(),
+          c1.getFamilyOffset(), fLen1, c2.getFamilyBuffer(), c2.getFamilyOffset(), fLen2);
       commonPrefix += fCommonPrefix;
       if (fCommonPrefix != fLen1) {
         return commonPrefix;
@@ -839,8 +1007,8 @@ public final class CellUtil {
     // Compare the Qualifiers
     int qLen1 = c1.getQualifierLength();
     int qLen2 = c2.getQualifierLength();
-    int qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
-        qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
+    int qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierBuffer(), c1.getQualifierOffset(),
+        qLen1, c2.getQualifierBuffer(), c2.getQualifierOffset(), qLen2);
     commonPrefix += qCommon;
     if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
       return commonPrefix;
@@ -887,4 +1055,37 @@ public final class CellUtil {
 
     return builder.toString();
   }
+
+  public static Cell createLastOnRow(Cell cell) {
+    if (cell.hasArray()) {
+      return KeyValueUtil.createLastOnRow(cell);
+    }
+    return new LastOnRowCell(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength());
+  }
+
+  public static Cell createFirstOnRow(Cell cell) {
+    if (cell.hasArray()) {
+      return KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(),
+          cell.getRowLength());
+    }
+    return new FirstOnRowCell(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength());
+  }
+
+  public static Cell createLastOnColumn(Cell cell) {
+    if (cell.hasArray()) {
+      return KeyValueUtil.createLastOnRowCol(cell);
+    }
+    return new LastOnColumnCell(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength(),
+        cell.getFamilyBuffer(), cell.getFamilyOffset(), cell.getFamilyLength(),
+        cell.getQualifierBuffer(), cell.getQualifierOffset(), cell.getQualifierLength());
+  }
+
+  public static Cell createFirstOnRowColTS(Cell c, long ts) {
+    if (c.hasArray()) {
+      return KeyValueUtil.createFirstOnRowColTS(c, ts);
+    }
+    return new FirstOnColumnTSCell(c.getRowBuffer(), c.getRowOffset(), c.getRowLength(),
+        c.getFamilyBuffer(), c.getFamilyOffset(), c.getFamilyLength(), c.getQualifierBuffer(),
+        c.getQualifierOffset(), c.getQualifierLength(), ts);
+  }
 }
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnColumnCell.java hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnColumnCell.java
new file mode 100644
index 0000000..c650e3f
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnColumnCell.java
@@ -0,0 +1,40 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.KeyValue.Type;
+
+public class FirstOnColumnCell extends LastOnColumnCell {
+
+  public FirstOnColumnCell(ByteBuffer row, int rOff, short rLen, ByteBuffer fam, int fOff,
+      byte fLen, ByteBuffer qual, int qOff, int qLen) {
+    super(row, rOff, rLen, fam, fOff, fLen, qual, qOff, qLen);
+  }
+
+  @Override
+  public long getTimestamp() {
+    return HConstants.LATEST_TIMESTAMP;
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return Type.Maximum.getCode();
+  }
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnColumnTSCell.java hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnColumnTSCell.java
new file mode 100644
index 0000000..7da48fb
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnColumnTSCell.java
@@ -0,0 +1,36 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+
+public class FirstOnColumnTSCell extends FirstOnColumnCell {
+
+  private final long ts;
+
+  public FirstOnColumnTSCell(ByteBuffer row, int rOff, short rLen, ByteBuffer fam, int fOff,
+      byte fLen, ByteBuffer qual, int qOff, int qLen, long ts) {
+    super(row, rOff, rLen, fam, fOff, fLen, qual, qOff, qLen);
+    this.ts = ts;
+  }
+
+  @Override
+  public long getTimestamp() {
+    return this.ts;
+  }
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnRowCell.java hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnRowCell.java
new file mode 100644
index 0000000..e6e65db
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/FirstOnRowCell.java
@@ -0,0 +1,39 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.KeyValue.Type;
+
+public class FirstOnRowCell extends LastOnRowCell {
+
+  public FirstOnRowCell(ByteBuffer row, int rOff, short rLen) {
+    super(row, rOff, rLen);
+  }
+
+  @Override
+  public long getTimestamp() {
+    return HConstants.LATEST_TIMESTAMP;
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return Type.Maximum.getCode();
+  }
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 19e251a..ed7473c 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase;
 
 import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH;
 
+import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Collections;
@@ -490,6 +491,8 @@ public final class HConstants {
    */
   public static final byte [] EMPTY_BYTE_ARRAY = new byte [0];
 
+  public static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(EMPTY_BYTE_ARRAY);
+
   /**
    * Used by scanners, etc when they want to start at the beginning of a region
    */
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index f265a38..a0982ea 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -39,6 +39,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.IOUtils;
@@ -79,7 +80,8 @@ import com.google.common.annotations.VisibleForTesting;
  * and actual tag bytes length.
  */
 @InterfaceAudience.Private
-public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, SettableTimestamp {
+public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, SettableTimestamp,
+    OutputStreamWritable {
   private static final ArrayList EMPTY_ARRAY_LIST = new ArrayList();
 
   static final Log LOG = LogFactory.getLog(KeyValue.class);
@@ -922,13 +924,18 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tagsLength > 0) {
       pos = Bytes.putAsShort(buffer, pos, tagsLength);
       for (Tag t : tags) {
-        pos = Bytes.putBytes(buffer, pos, t.getBuffer(), t.getOffset(), t.getLength());
+        if (t.hasArray()) {
+          pos = Bytes.putBytes(buffer, pos, t.getBytes(), t.getOffset(), t.getLength());
+        } else {
+          pos = ByteBufferUtils.copyFromBufferToByteArray(buffer, t.getBuffer(), t.getOffset(),
+              pos, t.getLength());
+        }
       }
     }
     return keyValueLength;
   }
 
-  private static void checkForTagsLength(int tagsLength) {
+  static void checkForTagsLength(int tagsLength) {
     if (tagsLength > MAX_TAGS_LENGTH) {
       throw new IllegalArgumentException("tagslength "+ tagsLength + " > " + MAX_TAGS_LENGTH);
     }
@@ -1047,7 +1054,12 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tagsLength > 0) {
       pos = Bytes.putAsShort(bytes, pos, tagsLength);
       for (Tag t : tags) {
-        pos = Bytes.putBytes(bytes, pos, t.getBuffer(), t.getOffset(), t.getLength());
+        if (t.hasArray()) {
+          pos = Bytes.putBytes(bytes, pos, t.getBytes(), t.getOffset(), t.getLength());
+        } else {
+          pos = ByteBufferUtils.copyFromBufferToByteArray(bytes, t.getBuffer(), t.getOffset(), pos,
+              t.getLength());
+        }
       }
     }
     return bytes;
@@ -1155,7 +1167,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tags != null) {
       List tagsString = new ArrayList();
       for (Tag t : tags) {
-        tagsString.add((t.getType()) + ":" +Bytes.toStringBinary(t.getValue()));
+        tagsString.add((t.getType()) + ":" +Bytes.toStringBinary(TagUtil.getTagValue(t)));
       }
       stringMap.put("tag", tagsString);
     }
@@ -1299,6 +1311,11 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     return bytes;
   }
 
+  @Override
+  public boolean hasArray() {
+    return true;
+  }
+
   /**
    * @return Row offset
    */
@@ -1594,7 +1611,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     if (tagsLength == 0) {
       return EMPTY_ARRAY_LIST;
     }
-    return Tag.asList(getTagsArray(), getTagsOffset(), tagsLength);
+    return TagUtil.asList(getTagsArray(), getTagsOffset(), tagsLength);
   }
 
   /**
@@ -1605,6 +1622,31 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     return bytes;
   }
 
+  @Override
+  public ByteBuffer getRowBuffer() {
+    return ByteBuffer.wrap(bytes);
+  }
+
+  @Override
+  public ByteBuffer getFamilyBuffer() {
+    return ByteBuffer.wrap(bytes);
+  }
+
+  @Override
+  public ByteBuffer getQualifierBuffer() {
+    return ByteBuffer.wrap(bytes);
+  }
+
+  @Override
+  public ByteBuffer getTagsBuffer() {
+    return ByteBuffer.wrap(bytes);
+  }
+
+  @Override
+  public ByteBuffer getValueBuffer() {
+    return ByteBuffer.wrap(bytes);
+  }
+
   /**
    * Creates a new KeyValue that only contains the key portion (the value is
    * set to be null).
@@ -1693,6 +1735,21 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     return result;
   }
 
+  public static int getDelimiter(final ByteBuffer b, int offset, final int length,
+      final int delimiter) {
+    if (b == null) {
+      throw new IllegalArgumentException("Passed buffer is null");
+    }
+    int result = -1;
+    for (int i = offset; i < length + offset; i++) {
+      if (ByteBufferUtils.getByte(b, i) == delimiter) {
+        result = i;
+        break;
+      }
+    }
+    return result;
+  }
+
   /**
    * Find index of passed delimiter walking from end of buffer backwards.
    * @param b
@@ -1714,6 +1771,21 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     return result;
   }
 
+  public static int getDelimiterInReverse(final ByteBuffer b, final int offset,
+      final int length, final int delimiter) {
+    if (b == null) {
+      throw new IllegalArgumentException("Passed buffer is null");
+    }
+    int result = -1;
+    for (int i = (offset + length) - 1; i >= offset; i--) {
+      if (b.get(i) == delimiter) {
+        result = i;
+        break;
+      }
+    }
+    return result;
+  }
+
   /**
    * A {@link KVComparator} for hbase:meta catalog table
    * {@link KeyValue}s.
@@ -1791,6 +1863,118 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
       return result;
     }
 
+    @Override
+    public int compareRows(ByteBuffer left, int loffset, int llength,
+        ByteBuffer right, int roffset, int rlength) {
+      int leftDelimiter = getDelimiter(left, loffset, llength,
+          HConstants.DELIMITER);
+      int rightDelimiter = getDelimiter(right, roffset, rlength,
+          HConstants.DELIMITER);
+      // Compare up to the delimiter
+      int lpart = (leftDelimiter < 0 ? llength :leftDelimiter - loffset);
+      int rpart = (rightDelimiter < 0 ? rlength :rightDelimiter - roffset);
+      int result = ByteBufferUtils.compareTo(left, loffset, lpart, right, roffset, rpart);
+      if (result != 0) {
+        return result;
+      } else {
+        if (leftDelimiter < 0 && rightDelimiter >= 0) {
+          return -1;
+        } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
+          return 1;
+        } else if (leftDelimiter < 0 && rightDelimiter < 0) {
+          return 0;
+        }
+      }
+      // Compare middle bit of the row.
+      // Move past delimiter
+      leftDelimiter++;
+      rightDelimiter++;
+      int leftFarDelimiter = getDelimiterInReverse(left, leftDelimiter,
+          llength - (leftDelimiter - loffset), HConstants.DELIMITER);
+      int rightFarDelimiter = getDelimiterInReverse(right,
+          rightDelimiter, rlength - (rightDelimiter - roffset),
+          HConstants.DELIMITER);
+      // Now compare middlesection of row.
+      lpart = (leftFarDelimiter < 0 ? llength + loffset: leftFarDelimiter) - leftDelimiter;
+      rpart = (rightFarDelimiter < 0 ? rlength + roffset: rightFarDelimiter)- rightDelimiter;
+      result = super.compareRows(left, leftDelimiter, lpart, right, rightDelimiter, rpart);
+      if (result != 0) {
+        return result;
+      }  else {
+        if (leftDelimiter < 0 && rightDelimiter >= 0) {
+          return -1;
+        } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
+          return 1;
+        } else if (leftDelimiter < 0 && rightDelimiter < 0) {
+          return 0;
+        }
+      }
+      // Compare last part of row, the rowid.
+      leftFarDelimiter++;
+      rightFarDelimiter++;
+      result = ByteBufferUtils.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
+          right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+      return result;
+    }
+
+    @Override
+    public int compareRows(byte [] left, int loffset, int llength,
+        ByteBuffer right, int roffset, int rlength) {
+      return -(compareRows(right, roffset, rlength,left, loffset, llength));
+    }
+    
+    @Override
+    public int compareRows(ByteBuffer left, int loffset, int llength, byte[] right,
+        int roffset, int rlength) {
+      int leftDelimiter = getDelimiter(left, loffset, llength, HConstants.DELIMITER);
+      int rightDelimiter = getDelimiter(right, roffset, rlength, HConstants.DELIMITER);
+      // Compare up to the delimiter
+      int lpart = (leftDelimiter < 0 ? llength : leftDelimiter - loffset);
+      int rpart = (rightDelimiter < 0 ? rlength : rightDelimiter - roffset);
+      int result = ByteBufferUtils.compareTo(left, loffset, lpart, right, roffset, rpart);
+      if (result != 0) {
+        return result;
+      } else {
+        if (leftDelimiter < 0 && rightDelimiter >= 0) {
+          return -1;
+        } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
+          return 1;
+        } else if (leftDelimiter < 0 && rightDelimiter < 0) {
+          return 0;
+        }
+      }
+      // Compare middle bit of the row.
+      // Move past delimiter
+      leftDelimiter++;
+      rightDelimiter++;
+      int leftFarDelimiter = getDelimiterInReverse(left, leftDelimiter, llength
+          - (leftDelimiter - loffset), HConstants.DELIMITER);
+      int rightFarDelimiter = getDelimiterInReverse(right, rightDelimiter, rlength
+          - (rightDelimiter - roffset), HConstants.DELIMITER);
+      // Now compare middlesection of row.
+      lpart = (leftFarDelimiter < 0 ? llength + loffset : leftFarDelimiter) - leftDelimiter;
+      rpart = (rightFarDelimiter < 0 ? rlength + roffset : rightFarDelimiter) - rightDelimiter;
+      result = super.compareRows(left, leftDelimiter, lpart, right, rightDelimiter, rpart);
+      if (result != 0) {
+        return result;
+      } else {
+        if (leftDelimiter < 0 && rightDelimiter >= 0) {
+          return -1;
+        } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
+          return 1;
+        } else if (leftDelimiter < 0 && rightDelimiter < 0) {
+          return 0;
+        }
+      }
+      // Compare last part of row, the rowid.
+      leftFarDelimiter++;
+      rightFarDelimiter++;
+      result = ByteBufferUtils.compareTo(left, leftFarDelimiter, llength
+          - (leftFarDelimiter - loffset), right, rightFarDelimiter, rlength
+          - (rightFarDelimiter - roffset));
+      return result;
+    }
+
     /**
      * Don't do any fancy Block Index splitting tricks.
      */
@@ -1820,13 +2004,20 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
      */
     @Override
     protected int compareRowKey(final Cell l, final Cell r) {
-      byte[] left = l.getRowArray();
-      int loffset = l.getRowOffset();
-      int llength = l.getRowLength();
-      byte[] right = r.getRowArray();
-      int roffset = r.getRowOffset();
-      int rlength = r.getRowLength();
-      return compareRows(left, loffset, llength, right, roffset, rlength);
+      if (l.hasArray()) {
+        if (r.hasArray()) {
+          return compareRows(l.getRowArray(), l.getRowOffset(), l.getRowLength(), r.getRowArray(),
+              r.getRowOffset(), r.getRowLength());
+        }
+        return compareRows(l.getRowArray(), l.getRowOffset(), l.getRowLength(), r.getRowBuffer(),
+            r.getRowOffset(), r.getRowLength());
+      }
+      if (r.hasArray()) {
+        return compareRows(l.getRowBuffer(), l.getRowOffset(), l.getRowLength(), r.getRowArray(),
+            r.getRowOffset(), r.getRowLength());
+      }
+      return compareRows(l.getRowBuffer(), l.getRowOffset(), l.getRowLength(), r.getRowBuffer(),
+          r.getRowOffset(), r.getRowLength());
     }
   }
 
@@ -1892,6 +2083,75 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
           rlength, rrowlength);
     }
 
+    /**
+     * Compares left to right assuming that left,loffset,llength and right,roffset,rlength are
+     * full KVs laid out in a flat byte[]s.
+     * @param left
+     * @param loffset
+     * @param llength
+     * @param right
+     * @param roffset
+     * @param rlength
+     * @return  0 if equal, <0 if left smaller, >0 if right smaller
+     */
+    public int compareFlatKey(ByteBuffer left, int loffset, int llength,
+        ByteBuffer right, int roffset, int rlength) {
+      // Compare row
+      short lrowlength = ByteBufferUtils.getShort(left, loffset);
+      short rrowlength = ByteBufferUtils.getShort(right, roffset);
+      int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
+          lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare the rest of the two KVs without making any assumptions about
+      // the common prefix. This function will not compare rows anyway, so we
+      // don't need to tell it that the common prefix includes the row.
+      return compareWithoutRow(0, left, loffset, llength, right, roffset,
+          rlength, rrowlength);
+    }
+
+    public int compareFlatKey(ByteBuffer left, int loffset, int llength,
+        byte[] right, int roffset, int rlength) {
+      // Compare row
+      short lrowlength = ByteBufferUtils.getShort(left, loffset);
+      short rrowlength = Bytes.toShort(right, roffset);
+      int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
+          lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare the rest of the two KVs without making any assumptions about
+      // the common prefix. This function will not compare rows anyway, so we
+      // don't need to tell it that the common prefix includes the row.
+      return compareWithoutRow(0, left, loffset, llength, right, roffset,
+          rlength, rrowlength);
+    }
+
+    public int compareFlatKey(ByteBuffer left, ByteBuffer right) {
+      return compareFlatKey(left, 0, left.limit(), right, 0, right.limit());
+    }
+
+    public int compareFlatKey(byte[] left, int loffset, int llength,
+        ByteBuffer right, int roffset, int rlength) {
+      // Compare row
+      short lrowlength =  Bytes.toShort(left, loffset);
+      short rrowlength = ByteBufferUtils.getShort(right, roffset);
+      int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
+          lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare the rest of the two KVs without making any assumptions about
+      // the common prefix. This function will not compare rows anyway, so we
+      // don't need to tell it that the common prefix includes the row.
+      return compareWithoutRow(0, left, loffset, llength, right, roffset,
+          rlength, rrowlength);
+    }
+
     public int compareFlatKey(byte[] left, byte[] right) {
       return compareFlatKey(left, 0, left.length, right, 0, right.length);
     }
@@ -1903,9 +2163,15 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
         byte[] col, int coff, int clen,
         long ts, byte type) {
 
-      int compare = compareRows(
-        cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
-        row, roff, rlen);
+      int compare = 0;
+      boolean hasArray = cell.hasArray();
+      if(hasArray) {
+        compare = compareRows(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), row,
+            roff, rlen);
+      } else {
+        compare = compareRows(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength(), row,
+            roff, rlen);
+      }
       if (compare != 0) {
         return compare;
       }
@@ -1923,15 +2189,23 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
         return -1;
       }
 
-      compare = compareFamilies(
-        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
-        fam, foff, flen);
+      if (hasArray) {
+        compare = compareFamilies(cell.getFamilyArray(), cell.getFamilyOffset(),
+            cell.getFamilyLength(), fam, foff, flen);
+      } else {
+        compare = compareFamilies(cell.getFamilyBuffer(), cell.getFamilyOffset(),
+            cell.getFamilyLength(), fam, foff, flen);
+      }
       if (compare != 0) {
         return compare;
       }
-      compare = compareColumns(
-        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
-        col, coff, clen);
+      if (hasArray) {
+        compare = compareColumns(cell.getQualifierArray(), cell.getQualifierOffset(),
+            cell.getQualifierLength(), col, coff, clen);
+      } else {
+        compare = compareColumns(cell.getQualifierBuffer(), cell.getQualifierOffset(),
+            cell.getQualifierLength(), col, coff, clen);
+      }
       if (compare != 0) {
         return compare;
       }
@@ -1947,24 +2221,160 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
       // everything.
       return (0xff & type) - (0xff & cell.getTypeByte());
     }
+    
+    // compare a key against row/fam/qual/ts/type
+    public int compareKey(Cell cell,
+        ByteBuffer row, int roff, int rlen,
+        ByteBuffer fam, int foff, int flen,
+        ByteBuffer col, int coff, int clen,
+        long ts, byte type) {
 
-    public int compareOnlyKeyPortion(Cell left, Cell right) {
-      return CellComparator.compare(left, right, true);
-    }
+      int compare = 0;
+      boolean hasArray = cell.hasArray();
+      if(hasArray) {
+        compare = compareRows(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), row,
+            roff, rlen);
+      } else {
+        compare = compareRows(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength(), row,
+            roff, rlen);
+      }
+      if (compare != 0) {
+        return compare;
+      }
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long). The
+      // "maximum" key type does not need this behavior.
+      if (cell.getFamilyLength() + cell.getQualifierLength() == 0
+          && cell.getTypeByte() == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (flen+clen == 0 && type == Type.Minimum.getCode()) {
+        return -1;
+      }
 
-    /**
-     * Compares the Key of a cell -- with fields being more significant in this order:
-     * rowkey, colfam/qual, timestamp, type, mvcc
-     */
-    @Override
-    public int compare(final Cell left, final Cell right) {
-      int compare = CellComparator.compare(left, right, false);
-      return compare;
-    }
+      if (hasArray) {
+        compare = compareFamilies(fam, foff, flen, cell.getFamilyArray(), cell.getFamilyOffset(),
+            cell.getFamilyLength());
+        compare = -compare;
+      } else {
+        compare = compareFamilies(cell.getFamilyBuffer(), cell.getFamilyOffset(),
+            cell.getFamilyLength(), fam, foff, flen);
+      }
+      if (compare != 0) {
+        return compare;
+      }
+      if (hasArray) {
+        compare = compareColumns( col, coff, clen, cell.getQualifierArray(), cell.getQualifierOffset(),
+            cell.getQualifierLength());
+        compare = -compare;
+      } else {
+        compare = compareColumns(cell.getQualifierBuffer(), cell.getQualifierOffset(),
+            cell.getQualifierLength(), col, coff, clen);
+      }
+      if (compare != 0) {
+        return compare;
+      }
+      // Next compare timestamps.
+      compare = compareTimestamps(cell.getTimestamp(), ts);
+      if (compare != 0) {
+        return compare;
+      }
 
-    public int compareTimestamps(final Cell left, final Cell right) {
-      return CellComparator.compareTimestamps(left, right);
-    }
+      // Compare types. Let the delete types sort ahead of puts; i.e. types
+      // of higher numbers sort before those of lesser numbers. Maximum (255)
+      // appears ahead of everything, and minimum (0) appears after
+      // everything.
+      return (0xff & type) - (0xff & cell.getTypeByte());
+    }
+    
+    // compare a key against row/fam/qual/ts/type
+    public int compareKey(Cell cell,
+        ByteBuffer row, int roff, int rlen,
+        ByteBuffer fam, int foff, int flen,
+        byte[] col, int coff, int clen,
+        long ts, byte type) {
+
+      int compare = 0;
+      boolean hasArray = cell.hasArray();
+      if(hasArray) {
+        compare = compareRows(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), row,
+            roff, rlen);
+      } else {
+        compare = compareRows(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength(), row,
+            roff, rlen);
+      }
+      if (compare != 0) {
+        return compare;
+      }
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long). The
+      // "maximum" key type does not need this behavior.
+      if (cell.getFamilyLength() + cell.getQualifierLength() == 0
+          && cell.getTypeByte() == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (flen+clen == 0 && type == Type.Minimum.getCode()) {
+        return -1;
+      }
+
+      if (hasArray) {
+        compare = compareFamilies(fam, foff, flen, cell.getFamilyArray(), cell.getFamilyOffset(),
+            cell.getFamilyLength());
+        compare = -compare;
+      } else {
+        compare = compareFamilies(cell.getFamilyBuffer(), cell.getFamilyOffset(),
+            cell.getFamilyLength(), fam, foff, flen);
+      }
+      if (compare != 0) {
+        return compare;
+      }
+      if (hasArray) {
+        compare = compareColumns( col, coff, clen, cell.getQualifierArray(), cell.getQualifierOffset(),
+            cell.getQualifierLength());
+        compare = -compare;
+      } else {
+        compare = compareColumns(cell.getQualifierBuffer(), cell.getQualifierOffset(),
+            cell.getQualifierLength(), col, coff, clen);
+      }
+      if (compare != 0) {
+        return compare;
+      }
+      // Next compare timestamps.
+      compare = compareTimestamps(cell.getTimestamp(), ts);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare types. Let the delete types sort ahead of puts; i.e. types
+      // of higher numbers sort before those of lesser numbers. Maximum (255)
+      // appears ahead of everything, and minimum (0) appears after
+      // everything.
+      return (0xff & type) - (0xff & cell.getTypeByte());
+    }
+
+    public int compareOnlyKeyPortion(Cell left, Cell right) {
+      return CellComparator.compare(left, right, true);
+    }
+
+    /**
+     * Compares the Key of a cell -- with fields being more significant in this order:
+     * rowkey, colfam/qual, timestamp, type, mvcc
+     */
+    @Override
+    public int compare(final Cell left, final Cell right) {
+      int compare = CellComparator.compare(left, right, false);
+      return compare;
+    }
+
+    public int compareTimestamps(final Cell left, final Cell right) {
+      return CellComparator.compareTimestamps(left, right);
+    }
 
     /**
      * @param left
@@ -1972,8 +2382,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
      * @return Result comparing rows.
      */
     public int compareRows(final Cell left, final Cell right) {
-      return compareRows(left.getRowArray(),left.getRowOffset(), left.getRowLength(),
-      right.getRowArray(), right.getRowOffset(), right.getRowLength());
+      return compareRowKey(left, right);
     }
 
     /**
@@ -1991,6 +2400,35 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
       return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
     }
 
+    public int compareRows(byte [] left, int loffset, int llength,
+        ByteBuffer right, int roffset, int rlength) {
+      return ByteBufferUtils.compareTo(left, loffset, llength, right, roffset, rlength);
+    }
+
+    public int compareRows(ByteBuffer left, int loffset, int llength,
+        byte[] right, int roffset, int rlength) {
+      return ByteBufferUtils.compareTo(left, loffset, llength, right, roffset, rlength);
+    }
+
+    public int compareRows(Cell leftCell, byte[] right, int roffset, int rlength) {
+      if (leftCell.hasArray()) {
+        return compareRows(leftCell.getRowArray(), leftCell.getRowOffset(),
+            leftCell.getRowLength(), right, roffset, rlength);
+      }
+      return compareRows(leftCell.getRowBuffer(), leftCell.getRowOffset(), leftCell.getRowLength(),
+          right, roffset, rlength);
+    }
+
+    public int compareRows(ByteBuffer left, int loffset, int llength,
+        ByteBuffer right, int roffset, int rlength) {
+      return ByteBufferUtils.compareTo(left, loffset, llength, right, roffset, rlength);
+    }
+    
+    // TODO can be avoided
+    public int compareRows(ByteBuffer left, ByteBuffer right) {
+      return ByteBufferUtils.compareTo(left, 0, left.capacity(), right, 0, right.capacity());
+    }
+
     int compareColumns(final Cell left, final short lrowlength, final Cell right,
         final short rrowlength) {
       return CellComparator.compareColumns(left, right);
@@ -2068,6 +2506,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
           roffset, rlength, lrowlength);
     }
 
+    // TODO  code duplication with 4 methods.
     /**
      * Compare columnFamily, qualifier, timestamp, and key type (everything
      * except the row). This method is used both in the normal comparator and
@@ -2160,17 +2599,284 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
       return (0xff & rtype) - (0xff & ltype);
     }
 
+    // TODO : Move these to utils classes
+    private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
+        int llength, ByteBuffer right, int roffset, int rlength, short rowlength) {
+      /***
+       * KeyValue Format and commonLength:
+       * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
+       * ------------------|-------commonLength--------|--------------
+       */
+      int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
+
+      // commonLength + TIMESTAMP_TYPE_SIZE
+      int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
+      // ColumnFamily + Qualifier length.
+      int lcolumnlength = llength - commonLengthWithTSAndType;
+      int rcolumnlength = rlength - commonLengthWithTSAndType;
+
+      byte ltype = left[loffset + (llength - 1)];
+      byte rtype = right.get(roffset + (rlength - 1));
+
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long). The
+      // "maximum" key type does not need this behavior.
+      if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
+        return -1;
+      }
+
+      int lfamilyoffset = commonLength + loffset;
+      int rfamilyoffset = commonLength + roffset;
+
+      // Column family length.
+      int lfamilylength = left[lfamilyoffset - 1];
+      int rfamilylength = right.get(rfamilyoffset - 1);
+      // If left family size is not equal to right family size, we need not
+      // compare the qualifiers.
+      boolean sameFamilySize = (lfamilylength == rfamilylength);
+      int common = 0;
+      if (commonPrefix > 0) {
+        common = Math.max(0, commonPrefix - commonLength);
+        if (!sameFamilySize) {
+          // Common should not be larger than Math.min(lfamilylength,
+          // rfamilylength).
+          common = Math.min(common, Math.min(lfamilylength, rfamilylength));
+        } else {
+          common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
+        }
+      }
+      if (!sameFamilySize) {
+        // comparing column family is enough.
+        return ByteBufferUtils.compareTo(left, lfamilyoffset + common, lfamilylength
+            - common, right, rfamilyoffset + common, rfamilylength - common);
+      }
+      // Compare family & qualifier together.
+      final int comparison = ByteBufferUtils.compareTo(left, lfamilyoffset + common,
+          lcolumnlength - common, right, rfamilyoffset + common,
+          rcolumnlength - common);
+      if (comparison != 0) {
+        return comparison;
+      }
+
+      ////
+      // Next compare timestamps.
+      long ltimestamp = Bytes.toLong(left,
+          loffset + (llength - TIMESTAMP_TYPE_SIZE));
+      long rtimestamp = ByteBufferUtils.getLong(right,
+          roffset + (rlength - TIMESTAMP_TYPE_SIZE));
+      int compare = compareTimestamps(ltimestamp, rtimestamp);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare types. Let the delete types sort ahead of puts; i.e. types
+      // of higher numbers sort before those of lesser numbers. Maximum (255)
+      // appears ahead of everything, and minimum (0) appears after
+      // everything.
+      return (0xff & rtype) - (0xff & ltype);
+    }
+    private int compareWithoutRow(int commonPrefix, ByteBuffer left, int loffset,
+        int llength, byte[] right, int roffset, int rlength, short rowlength) {
+      /***
+       * KeyValue Format and commonLength:
+       * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
+       * ------------------|-------commonLength--------|--------------
+       */
+      int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
+
+      // commonLength + TIMESTAMP_TYPE_SIZE
+      int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
+      // ColumnFamily + Qualifier length.
+      int lcolumnlength = llength - commonLengthWithTSAndType;
+      int rcolumnlength = rlength - commonLengthWithTSAndType;
+
+      byte ltype = left.get(loffset + (llength - 1));
+      byte rtype = right[roffset + (rlength - 1)];
+
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long). The
+      // "maximum" key type does not need this behavior.
+      if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
+        return -1;
+      }
+
+      int lfamilyoffset = commonLength + loffset;
+      int rfamilyoffset = commonLength + roffset;
+
+      // Column family length.
+      int lfamilylength = left.get(lfamilyoffset - 1);
+      int rfamilylength = right[rfamilyoffset - 1];
+      // If left family size is not equal to right family size, we need not
+      // compare the qualifiers.
+      boolean sameFamilySize = (lfamilylength == rfamilylength);
+      int common = 0;
+      if (commonPrefix > 0) {
+        common = Math.max(0, commonPrefix - commonLength);
+        if (!sameFamilySize) {
+          // Common should not be larger than Math.min(lfamilylength,
+          // rfamilylength).
+          common = Math.min(common, Math.min(lfamilylength, rfamilylength));
+        } else {
+          common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
+        }
+      }
+      if (!sameFamilySize) {
+        // comparing column family is enough.
+        return ByteBufferUtils.compareTo(left, lfamilyoffset + common, lfamilylength
+            - common, right, rfamilyoffset + common, rfamilylength - common);
+      }
+      // Compare family & qualifier together.
+      final int comparison = ByteBufferUtils.compareTo(left, lfamilyoffset + common,
+          lcolumnlength - common, right, rfamilyoffset + common,
+          rcolumnlength - common);
+      if (comparison != 0) {
+        return comparison;
+      }
+
+      ////
+      // Next compare timestamps.
+      long ltimestamp = ByteBufferUtils.getLong(left,
+          loffset + (llength - TIMESTAMP_TYPE_SIZE));
+      long rtimestamp = Bytes.toLong(right,
+          roffset + (rlength - TIMESTAMP_TYPE_SIZE));
+      int compare = compareTimestamps(ltimestamp, rtimestamp);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare types. Let the delete types sort ahead of puts; i.e. types
+      // of higher numbers sort before those of lesser numbers. Maximum (255)
+      // appears ahead of everything, and minimum (0) appears after
+      // everything.
+      return (0xff & rtype) - (0xff & ltype);
+    }
+    private int compareWithoutRow(int commonPrefix, ByteBuffer left, int loffset,
+        int llength, ByteBuffer right, int roffset, int rlength, short rowlength) {
+      /***
+       * KeyValue Format and commonLength:
+       * |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
+       * ------------------|-------commonLength--------|--------------
+       */
+      int commonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rowlength;
+
+      // commonLength + TIMESTAMP_TYPE_SIZE
+      int commonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + commonLength;
+      // ColumnFamily + Qualifier length.
+      int lcolumnlength = llength - commonLengthWithTSAndType;
+      int rcolumnlength = rlength - commonLengthWithTSAndType;
+
+      byte ltype = left.get(loffset + (llength - 1));
+      byte rtype = right.get(roffset + (rlength - 1));
+
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long). The
+      // "maximum" key type does not need this behavior.
+      if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
+        return -1;
+      }
+
+      int lfamilyoffset = commonLength + loffset;
+      int rfamilyoffset = commonLength + roffset;
+
+      // Column family length.
+      int lfamilylength = left.get(lfamilyoffset - 1);
+      int rfamilylength = right.get(rfamilyoffset - 1);
+      // If left family size is not equal to right family size, we need not
+      // compare the qualifiers.
+      boolean sameFamilySize = (lfamilylength == rfamilylength);
+      int common = 0;
+      if (commonPrefix > 0) {
+        common = Math.max(0, commonPrefix - commonLength);
+        if (!sameFamilySize) {
+          // Common should not be larger than Math.min(lfamilylength,
+          // rfamilylength).
+          common = Math.min(common, Math.min(lfamilylength, rfamilylength));
+        } else {
+          common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
+        }
+      }
+      if (!sameFamilySize) {
+        // comparing column family is enough.
+        return ByteBufferUtils.compareTo(left, lfamilyoffset + common, lfamilylength
+            - common, right, rfamilyoffset + common, rfamilylength - common);
+      }
+      // Compare family & qualifier together.
+      final int comparison = ByteBufferUtils.compareTo(left, lfamilyoffset + common,
+          lcolumnlength - common, right, rfamilyoffset + common,
+          rcolumnlength - common);
+      if (comparison != 0) {
+        return comparison;
+      }
+
+      ////
+      // Next compare timestamps.
+      long ltimestamp = ByteBufferUtils.getLong(left,
+          loffset + (llength - TIMESTAMP_TYPE_SIZE));
+      long rtimestamp = ByteBufferUtils.getLong(right,
+          roffset + (rlength - TIMESTAMP_TYPE_SIZE));
+      int compare = compareTimestamps(ltimestamp, rtimestamp);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare types. Let the delete types sort ahead of puts; i.e. types
+      // of higher numbers sort before those of lesser numbers. Maximum (255)
+      // appears ahead of everything, and minimum (0) appears after
+      // everything.
+      return (0xff & rtype) - (0xff & ltype);
+    }
     protected int compareFamilies(final byte[] left, final int loffset, final int lfamilylength,
         final byte[] right, final int roffset, final int rfamilylength) {
       int diff = Bytes.compareTo(left, loffset, lfamilylength, right, roffset, rfamilylength);
       return diff;
     }
+    
+    protected int compareFamilies(final ByteBuffer left, final int loffset,
+        final int lfamilylength, final byte[] right, final int roffset, final int rfamilylength) {
+      return ByteBufferUtils.compareTo(left, loffset, lfamilylength, right, roffset, rfamilylength);
+    }
+
+    protected int compareFamilies(final ByteBuffer left, final int loffset,
+        final int lfamilylength, final ByteBuffer right, final int roffset, final int rfamilylength) {
+      return ByteBufferUtils.compareTo(left, loffset, lfamilylength, right, roffset, rfamilylength);
+    }
 
     protected int compareColumns(final byte[] left, final int loffset, final int lquallength,
         final byte[] right, final int roffset, final int rquallength) {
       int diff = Bytes.compareTo(left, loffset, lquallength, right, roffset, rquallength);
       return diff;
     }
+    
+    protected int compareColumns(final ByteBuffer left, final int loffset, final int lquallength,
+        final byte[] right, final int roffset, final int rquallength) {
+      int diff = ByteBufferUtils.compareTo(left, loffset, lquallength, right, roffset, rquallength);
+      return diff;
+    }
+    
+    protected int compareColumns(final ByteBuffer left, final int loffset, final int lquallength,
+        final ByteBuffer right, final int roffset, final int rquallength) {
+      int diff = ByteBufferUtils.compareTo(left, loffset, lquallength, right, roffset, rquallength);
+      return diff;
+    }
+
     /**
      * Compares the row and column of two keyvalues for equality
      * @param left
@@ -2182,9 +2888,14 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
       short lrowlength = left.getRowLength();
       short rrowlength = right.getRowLength();
 
+      int lfamilylength = left.getFamilyLength();
+      int rfamilylength = right.getFamilyLength();
+
+      int lclength = left.getQualifierLength();
+      int rclength = right.getQualifierLength();
+
       // TsOffset = end of column data. just comparing Row+CF length of each
-      if ((left.getRowLength() + left.getFamilyLength() + left.getQualifierLength()) != (right
-          .getRowLength() + right.getFamilyLength() + right.getQualifierLength())) {
+      if ((lrowlength + lfamilylength + lclength) != (rrowlength + rfamilylength + rclength)) {
         return false;
       }
 
@@ -2194,17 +2905,14 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
 
       int lfoffset = left.getFamilyOffset();
       int rfoffset = right.getFamilyOffset();
-      int lclength = left.getQualifierLength();
-      int rclength = right.getQualifierLength();
-      int lfamilylength = left.getFamilyLength();
-      int rfamilylength = right.getFamilyLength();
-      int diff = compareFamilies(left.getFamilyArray(), lfoffset, lfamilylength,
-          right.getFamilyArray(), rfoffset, rfamilylength);
+ 
+      int diff = compareFamilies(left.getFamilyBuffer(), lfoffset, lfamilylength,
+          right.getFamilyBuffer(), rfoffset, rfamilylength);
       if (diff != 0) {
         return false;
       } else {
-        diff = compareColumns(left.getQualifierArray(), left.getQualifierOffset(), lclength,
-            right.getQualifierArray(), right.getQualifierOffset(), rclength);
+        diff = ByteBufferUtils.compareTo(left.getQualifierBuffer(), left.getQualifierOffset(), lclength,
+            right.getQualifierBuffer(), right.getQualifierOffset(), rclength);
         return diff == 0;
       }
     }
@@ -2228,11 +2936,11 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
      * @param rrowlength
      * @return True if rows match.
      */
-    private boolean matchingRows(final Cell left, final short lrowlength,
-        final Cell right, final short rrowlength) {
-      return lrowlength == rrowlength &&
-          matchingRows(left.getRowArray(), left.getRowOffset(), lrowlength,
-              right.getRowArray(), right.getRowOffset(), rrowlength);
+    private boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
+        final short rrowlength) {
+      return lrowlength == rrowlength
+          && ByteBufferUtils.equals(left.getRowBuffer(), left.getRowOffset(), lrowlength,
+              right.getRowBuffer(), right.getRowOffset(), rrowlength);
     }
 
     /**
@@ -2345,7 +3053,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
       super.clone();
       return new KVComparator();
     }
-
   }
 
   /**
@@ -2423,7 +3130,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
   public static KeyValue cloneAndAddTags(Cell c, List newTags) {
     List existingTags = null;
     if(c.getTagsLength() > 0) {
-      existingTags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+      existingTags = TagUtil.asList(c);
       existingTags.addAll(newTags);
     } else {
       existingTags = newTags;
@@ -2526,6 +3233,11 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     return length + Bytes.SIZEOF_INT;
   }
 
+  @Override
+  public long oswrite(OutputStream out, boolean withTags) throws IOException {
+    return oswrite(this, out, withTags);
+  }
+
   /**
    * Comparator that compares row component only of a KeyValue.
    */
@@ -2591,19 +3303,19 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     @Override
     @VisibleForTesting
     public int compareOnlyKeyPortion(Cell left, Cell right) {
-      int c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getRowArray(), left.getRowOffset(),
-          left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
+      int c = ByteBufferUtils.compareTo(left.getRowBuffer(), left.getRowOffset(),
+          left.getRowLength(), right.getRowBuffer(), right.getRowOffset(), right.getRowLength());
       if (c != 0) {
         return c;
       }
-      c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getFamilyArray(), left.getFamilyOffset(),
-          left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
+      c = ByteBufferUtils.compareTo(left.getFamilyBuffer(), left.getFamilyOffset(),
+          left.getFamilyLength(), right.getFamilyBuffer(), right.getFamilyOffset(),
           right.getFamilyLength());
       if (c != 0) {
         return c;
       }
-      c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getQualifierArray(), left.getQualifierOffset(),
-          left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
+      c = ByteBufferUtils.compareTo(left.getQualifierBuffer(), left.getQualifierOffset(),
+          left.getQualifierLength(), right.getQualifierBuffer(), right.getQualifierOffset(),
           right.getQualifierLength());
       if (c != 0) {
         return c;
@@ -2787,6 +3499,31 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
     }
 
     @Override
+    public ByteBuffer getRowBuffer() {
+      return ByteBuffer.wrap(this.bytes);
+    }
+
+    @Override
+    public ByteBuffer getFamilyBuffer() {
+      return ByteBuffer.wrap(this.bytes);
+    }
+
+    @Override
+    public ByteBuffer getQualifierBuffer() {
+      return ByteBuffer.wrap(this.bytes);
+    }
+
+    @Override
+    public ByteBuffer getTagsBuffer() {
+      return HConstants.EMPTY_BYTE_BUFFER;
+    };
+
+    @Override
+    public ByteBuffer getValueBuffer() {
+      return HConstants.EMPTY_BYTE_BUFFER;
+    }
+
+    @Override
     public String toString() {
       if (this.bytes == null || this.bytes.length == 0) {
         return "empty";
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index dde15bc..68afd29 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -26,6 +26,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -54,7 +55,7 @@ public class KeyValueUtil {
         cell.getValueLength(), cell.getTagsLength(), true);
   }
 
-  private static int length(short rlen, byte flen, int qlen, int vlen, int tlen, boolean withTags) {
+  public static int length(short rlen, byte flen, int qlen, int vlen, int tlen, boolean withTags) {
     if (withTags) {
       return (int) (KeyValue.getKeyValueDataStructureSize(rlen, flen, qlen, vlen, tlen));
     }
@@ -71,7 +72,7 @@ public class KeyValueUtil {
     return keyLength(cell.getRowLength(), cell.getFamilyLength(), cell.getQualifierLength());
   }
 
-  private static int keyLength(short rlen, byte flen, int qlen) {
+  public static int keyLength(short rlen, byte flen, int qlen) {
     return (int) KeyValue.getKeyDataStructureSize(rlen, flen, qlen);
   }
 
@@ -544,8 +545,8 @@ public class KeyValueUtil {
 
   public static void oswrite(final Cell cell, final OutputStream out, final boolean withTags)
       throws IOException {
-    if (cell instanceof KeyValue) {
-      KeyValue.oswrite((KeyValue) cell, out, withTags);
+    if (cell instanceof OutputStreamWritable) {
+      ((OutputStreamWritable) cell).oswrite(out, withTags);
     } else {
       short rlen = cell.getRowLength();
       byte flen = cell.getFamilyLength();
@@ -561,26 +562,28 @@ public class KeyValueUtil {
       StreamUtils.writeInt(out, vlen);
       // Write rowkey - 2 bytes rk length followed by rowkey bytes
       StreamUtils.writeShort(out, rlen);
-      out.write(cell.getRowArray(), cell.getRowOffset(), rlen);
+      ByteBufferUtils.writeToStream(out, cell.getRowBuffer(), cell.getRowOffset(), rlen);
       // Write cf - 1 byte of cf length followed by the family bytes
       out.write(flen);
-      out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flen);
+      ByteBufferUtils.writeToStream(out, cell.getFamilyBuffer(), cell.getFamilyOffset(), flen);
       // write qualifier
-      out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
+      ByteBufferUtils
+          .writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset(), qlen);
       // write timestamp
       StreamUtils.writeLong(out, cell.getTimestamp());
       // write the type
       out.write(cell.getTypeByte());
       // write value
-      out.write(cell.getValueArray(), cell.getValueOffset(), vlen);
+      ByteBufferUtils.writeToStream(out, cell.getValueBuffer(), cell.getValueOffset(), vlen);
       // write tags if we have to
       if (withTags) {
         // 2 bytes tags length followed by tags bytes
-        // tags length is serialized with 2 bytes only(short way) even if the type is int. As this
+        // tags length is serialized with 2 bytes only(short way) even if the
+        // type is int. As this
         // is non -ve numbers, we save the sign bit. See HBASE-11437
         out.write((byte) (0xff & (tlen >> 8)));
         out.write((byte) (0xff & tlen));
-        out.write(cell.getTagsArray(), cell.getTagsOffset(), tlen);
+        ByteBufferUtils.writeToStream(out, cell.getTagsBuffer(), cell.getTagsOffset(), tlen);
       }
     }
   }
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/LastOnColumnCell.java hbase-common/src/main/java/org/apache/hadoop/hbase/LastOnColumnCell.java
new file mode 100644
index 0000000..852bed7
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/LastOnColumnCell.java
@@ -0,0 +1,194 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.KeyValue.Type;
+
+// This is backed by a buffer
+public class LastOnColumnCell implements Cell {
+
+  private final ByteBuffer row, fam, qual;
+  private final int rOff, fOff, qOff,qLen;
+  private final short rLen;
+  private final byte fLen;
+
+  public LastOnColumnCell(ByteBuffer row, int rOff, short rLen, ByteBuffer fam, int fOff,
+      byte fLen, ByteBuffer qual, int qOff, int qLen) {
+    this.row = row;
+    this.rOff = rOff;
+    this.rLen = rLen;
+    this.fam = fam;
+    this.fOff = fOff;
+    this.fLen = fLen;
+    this.qual = qual;
+    this.qOff = qOff;
+    this.qLen = qLen;
+  }
+
+  @Override
+  public byte[] getRowArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getRowOffset() {
+    return this.rOff;
+  }
+
+  @Override
+  public short getRowLength() {
+    return this.rLen;
+  }
+
+  @Override
+  public byte[] getFamilyArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    return this.fOff;
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return this.fLen;
+  }
+
+  @Override
+  public byte[] getQualifierArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    return this.qOff;
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return this.qLen;
+  }
+
+  @Override
+  public long getTimestamp() {
+    return HConstants.OLDEST_TIMESTAMP;
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return Type.Minimum.getCode();
+  }
+
+  @Override
+  @Deprecated
+  public long getMvccVersion() {
+    return getSequenceId();
+  }
+
+  @Override
+  public long getSequenceId() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getValueOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getValueLength() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getTagsArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getTagsOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getTagsLength() {
+    return 0;
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getValue() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getFamily() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getQualifier() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getRow() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public boolean hasArray() {
+    return false;
+  }
+
+  @Override
+  public ByteBuffer getRowBuffer() {
+    return this.row;
+  }
+
+  @Override
+  public ByteBuffer getFamilyBuffer() {
+    return this.fam;
+  }
+
+  @Override
+  public ByteBuffer getQualifierBuffer() {
+    return this.qual;
+  }
+
+  @Override
+  public ByteBuffer getValueBuffer() {
+    return HConstants.EMPTY_BYTE_BUFFER;
+  }
+
+  @Override
+  public ByteBuffer getTagsBuffer() {
+    return HConstants.EMPTY_BYTE_BUFFER;
+  }
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/LastOnRowCell.java hbase-common/src/main/java/org/apache/hadoop/hbase/LastOnRowCell.java
new file mode 100644
index 0000000..98e90a3
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/LastOnRowCell.java
@@ -0,0 +1,186 @@
+/**
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.KeyValue.Type;
+
+// This is backed by a buffer
+public class LastOnRowCell implements Cell {
+
+  private final ByteBuffer row;
+  private final int rOff;
+  private final short rLen;
+
+  public LastOnRowCell(ByteBuffer row, int rOff, short rLen) {
+    this.row = row;
+    this.rOff = rOff;
+    this.rLen = rLen;
+  }
+
+  @Override
+  public byte[] getRowArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getRowOffset() {
+    return this.rOff;
+  }
+
+  @Override
+  public short getRowLength() {
+    return this.rLen;
+  }
+
+  @Override
+  public byte[] getFamilyArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    return 0;
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getQualifierArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return 0;
+  }
+
+  @Override
+  public long getTimestamp() {
+    return HConstants.OLDEST_TIMESTAMP;
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return Type.Minimum.getCode();
+  }
+
+  @Override
+  @Deprecated
+  public long getMvccVersion() {
+    return getSequenceId();
+  }
+
+  @Override
+  public long getSequenceId() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getValueOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getValueLength() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getTagsArray() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public int getTagsOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getTagsLength() {
+    return 0;
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getValue() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getFamily() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getQualifier() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  @Deprecated
+  public byte[] getRow() {
+    throw new UnsupportedOperationException("byte[] not supported");
+  }
+
+  @Override
+  public boolean hasArray() {
+    return false;
+  }
+
+  @Override
+  public ByteBuffer getRowBuffer() {
+    return this.row;
+  }
+
+  @Override
+  public ByteBuffer getFamilyBuffer() {
+    return HConstants.EMPTY_BYTE_BUFFER;
+  }
+
+  @Override
+  public ByteBuffer getQualifierBuffer() {
+    return HConstants.EMPTY_BYTE_BUFFER;
+  }
+
+  @Override
+  public ByteBuffer getValueBuffer() {
+    return HConstants.EMPTY_BYTE_BUFFER;
+  }
+
+  @Override
+  public ByteBuffer getTagsBuffer() {
+    return HConstants.EMPTY_BYTE_BUFFER;
+  }
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/OutputStreamWritable.java hbase-common/src/main/java/org/apache/hadoop/hbase/OutputStreamWritable.java
new file mode 100644
index 0000000..9203e79
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/OutputStreamWritable.java
@@ -0,0 +1,26 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public interface OutputStreamWritable {
+
+  long oswrite(OutputStream out, boolean withTags) throws IOException;
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
index 2e7314d..b9190a0 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
@@ -19,211 +19,55 @@
  */
 package org.apache.hadoop.hbase;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
+
 /**
- * Tags are part of cells and helps to add metadata about the KVs.
- * Metadata could be ACLs per cells, visibility labels, etc.
+ * Tags are part of cells and helps to add metadata about the KVs. Metadata could be ACLs per cells,
+ * visibility labels, etc.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class Tag {
-  public final static int TYPE_LENGTH_SIZE = Bytes.SIZEOF_BYTE;
-  public final static int TAG_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
-  public final static int INFRASTRUCTURE_SIZE = TYPE_LENGTH_SIZE + TAG_LENGTH_SIZE;
-  public static final int MAX_TAG_LENGTH = (2 * Short.MAX_VALUE) + 1 - TAG_LENGTH_SIZE;
-
-  private final byte type;
-  private final byte[] bytes;
-  private int offset = 0;
-  private int length = 0;
-
-  // The special tag will write the length of each tag and that will be
-  // followed by the type and then the actual tag.
-  // So every time the length part is parsed we need to add + 1 byte to it to
-  // get the type and then get the actual tag.
-  public Tag(byte tagType, String tag) {
-    this(tagType, Bytes.toBytes(tag));
-  }
-
-  /**
-   * @param tagType
-   * @param tag
-   */
-  public Tag(byte tagType, byte[] tag) {
-    /**
-     * Format for a tag :  taglength is serialized
-     * using 2 bytes only but as this will be unsigned, we can have max taglength of
-     * (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
-     */
-    int tagLength = tag.length + TYPE_LENGTH_SIZE;
-    if (tagLength > MAX_TAG_LENGTH) {
-      throw new IllegalArgumentException(
-          "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH);
-    }
-    length = TAG_LENGTH_SIZE + tagLength;
-    bytes = new byte[length];
-    int pos = Bytes.putAsShort(bytes, 0, tagLength);
-    pos = Bytes.putByte(bytes, pos, tagType);
-    Bytes.putBytes(bytes, pos, tag, 0, tag.length);
-    this.type = tagType;
-  }
-
-  /**
-   * Creates a Tag from the specified byte array and offset. Presumes
-   * bytes content starting at offset is formatted as
-   * a Tag blob.
-   * The bytes to include the tag type, tag length and actual tag bytes.
-   * @param bytes
-   *          byte array
-   * @param offset
-   *          offset to start of Tag
-   */
-  public Tag(byte[] bytes, int offset) {
-    this(bytes, offset, getLength(bytes, offset));
-  }
-
-  private static int getLength(byte[] bytes, int offset) {
-    return TAG_LENGTH_SIZE + Bytes.readAsInt(bytes, offset, TAG_LENGTH_SIZE);
-  }
-
-  /**
-   * Creates a Tag from the specified byte array, starting at offset, and for length
-   * length. Presumes bytes content starting at offset is
-   * formatted as a Tag blob.
-   * @param bytes
-   *          byte array
-   * @param offset
-   *          offset to start of the Tag
-   * @param length
-   *          length of the Tag
-   */
-  public Tag(byte[] bytes, int offset, int length) {
-    if (length > MAX_TAG_LENGTH) {
-      throw new IllegalArgumentException(
-          "Invalid tag data being passed. Its length can not exceed " + MAX_TAG_LENGTH);
-    }
-    this.bytes = bytes;
-    this.offset = offset;
-    this.length = length;
-    this.type = bytes[offset + TAG_LENGTH_SIZE];
-  }
+public interface Tag {
+  int TYPE_LENGTH_SIZE = Bytes.SIZEOF_BYTE;
+  int TAG_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
+  int INFRASTRUCTURE_SIZE = TYPE_LENGTH_SIZE + TAG_LENGTH_SIZE;
+  int MAX_TAG_LENGTH = (2 * Short.MAX_VALUE) + 1 - TAG_LENGTH_SIZE;
 
   /**
    * @return The byte array backing this Tag.
    */
-  public byte[] getBuffer() {
-    return this.bytes;
-  }
+  byte[] getBytes();
+
+  ByteBuffer getBuffer();
 
   /**
    * @return the tag type
    */
-  public byte getType() {
-    return this.type;
-  }
+  byte getType();
 
   /**
    * @return Length of actual tag bytes within the backed buffer
    */
-  public int getTagLength() {
-    return this.length - INFRASTRUCTURE_SIZE;
-  }
+  int getTagLength();
 
   /**
    * @return Offset of actual tag bytes within the backed buffer
    */
-  public int getTagOffset() {
-    return this.offset + INFRASTRUCTURE_SIZE;
-  }
-
-  /**
-   * Returns tag value in a new byte array.
-   * Primarily for use client-side. If server-side, use
-   * {@link #getBuffer()} with appropriate {@link #getTagOffset()} and {@link #getTagLength()}
-   * instead to save on allocations.
-   * @return tag value in a new byte array.
-   */
-  public byte[] getValue() {
-    int tagLength = getTagLength();
-    byte[] tag = new byte[tagLength];
-    Bytes.putBytes(tag, 0, bytes, getTagOffset(), tagLength);
-    return tag;
-  }
-
-  /**
-   * Creates the list of tags from the byte array b. Expected that b is in the
-   * expected tag format
-   * @param b
-   * @param offset
-   * @param length
-   * @return List of tags
-   */
-  public static List asList(byte[] b, int offset, int length) {
-    List tags = new ArrayList();
-    int pos = offset;
-    while (pos < offset + length) {
-      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
-      tags.add(new Tag(b, pos, tagLen + TAG_LENGTH_SIZE));
-      pos += TAG_LENGTH_SIZE + tagLen;
-    }
-    return tags;
-  }
-
-  /**
-   * Write a list of tags into a byte array
-   * @param tags
-   * @return the serialized tag data as bytes
-   */
-  public static byte[] fromList(List tags) {
-    int length = 0;
-    for (Tag tag: tags) {
-      length += tag.length;
-    }
-    byte[] b = new byte[length];
-    int pos = 0;
-    for (Tag tag: tags) {
-      System.arraycopy(tag.bytes, tag.offset, b, pos, tag.length);
-      pos += tag.length;
-    }
-    return b;
-  }
-
-  /**
-   * Retrieve the first tag from the tags byte array matching the passed in tag type
-   * @param b
-   * @param offset
-   * @param length
-   * @param type
-   * @return null if there is no tag of the passed in tag type
-   */
-  public static Tag getTag(byte[] b, int offset, int length, byte type) {
-    int pos = offset;
-    while (pos < offset + length) {
-      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
-      if(b[pos + TAG_LENGTH_SIZE] == type) {
-        return new Tag(b, pos, tagLen + TAG_LENGTH_SIZE);
-      }
-      pos += TAG_LENGTH_SIZE + tagLen;
-    }
-    return null;
-  }
+  int getTagOffset();
 
   /**
    * Returns the total length of the entire tag entity
    */
-  int getLength() {
-    return this.length;
-  }
+  int getLength();
 
   /**
    * Returns the offset of the entire tag entity
    */
-  int getOffset() {
-    return this.offset;
-  }
+  int getOffset();
+
+  boolean hasArray();
 }
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
new file mode 100644
index 0000000..f98057d
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
@@ -0,0 +1,146 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class TagUtil {
+
+  /**
+   * Creates the list of tags from the byte array b. Expected that b is in the expected tag format
+   * 
+   * @param b
+   * @param offset
+   * @param length
+   * @return List of tags
+   */
+  public static List asList(byte[] b, int offset, int length) {
+    List tags = new ArrayList();
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
+      tags.add(new ArrayBackedTag(b, pos, tagLen + TAG_LENGTH_SIZE));
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return tags;
+  }
+
+  public static List asList(ByteBuffer b, int offset, int length) {
+    List tags = new ArrayList();
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen = ByteBufferUtils.getAsInt(b, pos, TAG_LENGTH_SIZE);
+      tags.add(new BufferBackedTag(b, pos, tagLen + TAG_LENGTH_SIZE));
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return tags;
+  }
+
+  public static List asList(Cell c) {
+    if (c.hasArray()) {
+      return asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    }
+    return asList(c.getTagsBuffer(), c.getTagsOffset(), c.getTagsLength());
+  }
+
+  /**
+   * Write a list of tags into a byte array
+   * 
+   * @param tags
+   * @return the serialized tag data as bytes
+   */
+  public static byte[] fromList(List tags) {
+    int length = 0;
+    for (Tag tag : tags) {
+      length += tag.getLength();
+    }
+    byte[] b = new byte[length];
+    int pos = 0;
+    for (Tag tag : tags) {
+      if (tag.hasArray()) {
+        System.arraycopy(tag.getBytes(), tag.getOffset(), b, pos, tag.getLength());
+      } else {
+        ByteBufferUtils.copyFromBufferToByteArray(b, tag.getBuffer(), tag.getOffset(), pos,
+            tag.getLength());
+      }
+      pos += tag.getLength();
+    }
+    return b;
+  }
+
+  /**
+   * Retrieve the first tag from the tags byte array matching the passed in tag type
+   * 
+   * @param b
+   * @param offset
+   * @param length
+   * @param type
+   * @return null if there is no tag of the passed in tag type
+   */
+  public static Tag getTag(byte[] b, int offset, int length, byte type) {
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
+      if (b[pos + TAG_LENGTH_SIZE] == type) {
+        return new ArrayBackedTag(b, pos, tagLen + TAG_LENGTH_SIZE);
+      }
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return null;
+  }
+
+  public static Tag getTag(ByteBuffer b, int offset, int length, byte type) {
+    int pos = offset;
+    while (pos < offset + length) {
+      int tagLen = ByteBufferUtils.getAsInt(b, pos, TAG_LENGTH_SIZE);
+      if (ByteBufferUtils.getByte(b, pos + TAG_LENGTH_SIZE) == type) {
+        return new BufferBackedTag(b, pos, tagLen + TAG_LENGTH_SIZE);
+      }
+      pos += TAG_LENGTH_SIZE + tagLen;
+    }
+    return null;
+  }
+
+  public static Tag getTag(Cell c, byte type) {
+    if (c.hasArray()) {
+      return getTag(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength(), type);
+    }
+    return getTag(c.getTagsBuffer(), c.getTagsOffset(), c.getTagsLength(), type);
+  }
+
+  public static byte[] getTagValue(Tag tag) {
+    int tagLength = tag.getTagLength();
+    byte[] tagBytes = new byte[tagLength];
+    if (tag.hasArray()) {
+      Bytes.putBytes(tagBytes, 0, tag.getBytes(), tag.getTagOffset(), tagLength);
+    } else {
+      ByteBufferUtils.copyFromBufferToByteArray(tagBytes, tag.getBuffer(), tag.getTagOffset(), 0,
+          tagLength);
+    }
+    return tagBytes;
+  }
+}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java
index 1530ccd..64d9a0c 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * Please note that the reads will cause position movement on wrapped ByteBuffer.
  */
 @InterfaceAudience.Private
+// TODO read using BBUtil here also.  Do Unsafe read as much as we can
 public class ByteBufferInputStream extends InputStream {
 
   private ByteBuffer buf;
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
index af12113..1525510 100644
--- hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferOutputStream.java
@@ -27,6 +27,7 @@ import java.nio.channels.WritableByteChannel;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -37,11 +38,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 public class ByteBufferOutputStream extends OutputStream {
 
   protected ByteBuffer buf;
-
+  
   public ByteBufferOutputStream(int capacity) {
     this(capacity, false);
   }
-
   public ByteBufferOutputStream(int capacity, boolean useDirectByteBuffer) {
     this(allocate(capacity, useDirectByteBuffer));
   }
@@ -61,14 +61,14 @@ public class ByteBufferOutputStream extends OutputStream {
     this.buf.clear();
   }
 
-  public int size() {
-    return buf.position();
-  }
-
   private static ByteBuffer allocate(final int capacity, final boolean useDirectByteBuffer) {
     return useDirectByteBuffer? ByteBuffer.allocateDirect(capacity): ByteBuffer.allocate(capacity);
   }
 
+  public int size() {
+    return buf.position();
+  }
+
   /**
    * This flips the underlying BB so be sure to use it _last_!
    * @return ByteBuffer
@@ -85,6 +85,7 @@ public class ByteBufferOutputStream extends OutputStream {
       int newSize = (int)Math.min((((long)buf.capacity()) * 2),
           (long)(Integer.MAX_VALUE));
       newSize = Math.max(newSize, buf.position() + extra);
+
       ByteBuffer newBuf = allocate(newSize, buf.isDirect());
       buf.flip();
       newBuf.put(buf);
@@ -127,6 +128,20 @@ public class ByteBufferOutputStream extends OutputStream {
 
     buf.put(b, off, len);
   }
+  
+  public void write(ByteBuffer b, int off, int len) throws IOException {
+    checkSizeAndGrow(len);
+    ByteBufferUtils.copyFromBufferToBuffer(buf, b, off, len);
+  }
+
+  /**
+   * Write given int (all 4 bytes) to the underlying stream.
+   * @param b
+   */
+  public void writeInt(int b){
+    checkSizeAndGrow(Bytes.SIZEOF_INT);
+    this.buf.putInt(b);// TODO use BBUtil?
+  }
 
   @Override
   public void flush() throws IOException {
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/MultiByteBufferInputStream.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/MultiByteBufferInputStream.java
new file mode 100644
index 0000000..dd77d6a
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/MultiByteBufferInputStream.java
@@ -0,0 +1,101 @@
+/**
+ * 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.io;
+
+import java.io.InputStream;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.MultiByteBuffer;
+
+/**
+ * Not thread safe!
+ * 

+ * Please note that the reads will cause position movement on wrapped ByteBuffer. + */ +@InterfaceAudience.Private +public class MultiByteBufferInputStream extends InputStream { + + private MultiByteBuffer buf; + + public MultiByteBufferInputStream(MultiByteBuffer buf) { + this.buf = buf; + } + + /** + * Reads the next byte of data from this input stream. The value byte is returned as an + * int in the range 0 to 255. If no byte is available + * because the end of the stream has been reached, the value -1 is returned. + * @return the next byte of data, or -1 if the end of the stream has been reached. + */ + public int read() { + if (this.buf.hasRemaining()) { + return (this.buf.get() & 0xff); + } + return -1; + } + + /** + * Reads up to next len bytes of data from buffer into passed array(starting from + * given offset). + * @param b the array into which the data is read. + * @param off the start offset in the destination array b + * @param len the maximum number of bytes to read. + * @return the total number of bytes actually read into the buffer, or -1 if not even + * 1 byte can be read because the end of the stream has been reached. + */ + public int read(byte b[], int off, int len) { + int avail = available(); + if (avail <= 0) { + return -1; + } + + if (len > avail) { + len = avail; + } + if (len <= 0) { + return 0; + } + + this.buf.get(b, off, len); + return len; + } + + /** + * Skips n bytes of input from this input stream. Fewer bytes might be skipped if the + * end of the input stream is reached. The actual number k of bytes to be skipped is + * equal to the smaller of n and remaining bytes in the stream. + * @param n the number of bytes to be skipped. + * @return the actual number of bytes skipped. + */ + public long skip(long n) { + long k = Math.min(n, available()); + if (k < 0) { + k = 0; + } + this.buf.skip((int) k); + return k; + } + + /** + * @return the number of remaining bytes that can be read (or skipped + * over) from this input stream. + */ + public int available() { + return this.buf.remaining(); + } +} diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/TagCompressionContext.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/TagCompressionContext.java index 26e7b50..19be849 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/TagCompressionContext.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/TagCompressionContext.java @@ -74,21 +74,16 @@ public class TagCompressionContext { } } - /** - * Compress tags one by one and writes to the OutputStream. - * @param out Stream to which the compressed tags to be written - * @param in Source buffer where tags are available - * @param length Length of all tag bytes - * @throws IOException - */ - public void compressTags(OutputStream out, ByteBuffer in, int length) throws IOException { - if (in.hasArray()) { - compressTags(out, in.array(), in.arrayOffset() + in.position(), length); - ByteBufferUtils.skip(in, length); - } else { - byte[] tagBuf = new byte[length]; - in.get(tagBuf); - compressTags(out, tagBuf, 0, length); + public void compressTags(OutputStream out, ByteBuffer in, int offset, int length) + throws IOException { + int pos = offset; + int endOffset = pos + length; + assert pos < endOffset; + while (pos < endOffset) { + int tagLen = ByteBufferUtils.getAsInt(in, pos, Tag.TAG_LENGTH_SIZE); + pos += Tag.TAG_LENGTH_SIZE; + write(in, pos, tagLen, out); + pos += tagLen; } } @@ -191,4 +186,21 @@ public class TagCompressionContext { StreamUtils.writeShort(out, dictIdx); } } + + private void write(ByteBuffer data, int offset, int length, OutputStream out) throws IOException { + // TODO avoid need to copy + byte[] temp = new byte[length]; + ByteBufferUtils.copyFromBufferToByteArray(temp, data, offset, 0, length); + short dictIdx = Dictionary.NOT_IN_DICTIONARY; + if (tagDict != null) { + dictIdx = tagDict.findEntry(temp, 0, length); + } + if (dictIdx == Dictionary.NOT_IN_DICTIONARY) { + out.write(Dictionary.NOT_IN_DICTIONARY); + StreamUtils.writeRawVInt32(out, length); + out.write(temp, 0, length); + } else { + StreamUtils.writeShort(out, dictIdx); + } + } } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java index be8c192..f82d8ea 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.io.encoding; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.nio.ByteBuffer; import org.apache.hadoop.hbase.Cell; @@ -30,13 +31,16 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.OutputStreamWritable; import org.apache.hadoop.hbase.SettableSequenceId; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.ByteBufferOutputStream; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.TagCompressionContext; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.util.LRUDictionary; +import org.apache.hadoop.hbase.io.util.StreamUtils; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; @@ -79,7 +83,9 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { return internalDecodeKeyValues(source, 0, 0, decodingCtx); } + // Need not impl BBBackedCell. Let the cloned seeker state impl it protected static class SeekerState implements Cell { + // This should referencing directly to the hfileblock protected ByteBuffer currentBuffer; protected TagCompressionContext tagCompressionContext; protected int valueOffset = -1; @@ -97,7 +103,10 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { protected long memstoreTS; protected int nextKvOffset; - protected KeyValue.KeyOnlyKeyValue currentKey = new KeyValue.KeyOnlyKeyValue(); + // Going with BRBackedKV is that we need to do lot of changes in the algos impl. + // Finally we could wrap this inside a BB + protected KeyValue.KeyOnlyKeyValue currentKey = + new KeyValue.KeyOnlyKeyValue(); protected boolean isValid() { return valueOffset != -1; @@ -154,7 +163,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { // Note: we can only call isValid before we override our state, so this // comes before all the assignments at the end of this method. System.arraycopy(nextState.keyBuffer, 0, keyBuffer, 0, - nextState.keyLength); + nextState.keyLength); } else { // don't copy the common prefix between this key and the previous one System.arraycopy(nextState.keyBuffer, nextState.lastCommonPrefix, @@ -174,10 +183,15 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { tagCompressionContext = nextState.tagCompressionContext; } } + + @Override + public boolean hasArray() { + return false; + } @Override public byte[] getRowArray() { - return currentKey.getRowArray(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override @@ -192,7 +206,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public byte[] getFamilyArray() { - return currentKey.getFamilyArray(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override @@ -207,7 +221,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public byte[] getQualifierArray() { - return currentKey.getQualifierArray(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override @@ -242,12 +256,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public byte[] getValueArray() { - return currentBuffer.array(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override public int getValueOffset() { - return currentBuffer.arrayOffset() + valueOffset; + return valueOffset; } @Override @@ -257,10 +271,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public byte[] getTagsArray() { - if (tagCompressionContext != null) { - return tagsBuffer; - } - return currentBuffer.array(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override @@ -268,7 +279,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { if (tagCompressionContext != null) { return 0; } - return currentBuffer.arrayOffset() + tagsOffset; + return tagsOffset; } @Override @@ -313,6 +324,34 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { currentKey.getTimestamp(), currentKey.getTypeByte(), valueLength, valueOffset, memstoreTS, tagsOffset, tagsLength, tagCompressionContext, tagsBuffer); } + + @Override + public ByteBuffer getRowBuffer() { + return currentKey.getRowBuffer(); + } + + @Override + public ByteBuffer getFamilyBuffer() { + return currentKey.getFamilyBuffer(); + } + + @Override + public ByteBuffer getQualifierBuffer() { + return currentKey.getQualifierBuffer(); + } + + @Override + public ByteBuffer getValueBuffer() { + return currentBuffer; + } + + @Override + public ByteBuffer getTagsBuffer() { + if (tagCompressionContext != null) { + return (ByteBuffer.wrap(tagsBuffer)); + } + return currentBuffer; + } } /** @@ -325,11 +364,13 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { // there. So this has to be an instance of SettableSequenceId. SeekerState need not be // SettableSequenceId as we never return that to top layers. When we have to, we make // ClonedSeekerState from it. - protected static class ClonedSeekerState implements Cell, HeapSize, SettableSequenceId { + protected static class ClonedSeekerState implements Cell, HeapSize, SettableSequenceId, + OutputStreamWritable { private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (4 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT) - + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (2 * ClassSize.ARRAY)); - private byte[] keyOnlyBuffer; + + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (2 * ClassSize.BYTE_BUFFER)); + private ByteBuffer keyOnlyBuffer; + // This will refer to direct HFileBlock private ByteBuffer currentBuffer; private short rowLength; private int familyOffset; @@ -342,7 +383,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { private int valueLength; private int tagsLength; private int tagsOffset; - private byte[] cloneTagsBuffer; + private ByteBuffer cloneTagsBuffer; private long seqId; private TagCompressionContext tagCompressionContext; @@ -352,7 +393,9 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { int tagsOffset, int tagsLength, TagCompressionContext tagCompressionContext, byte[] tagsBuffer) { this.currentBuffer = currentBuffer; - keyOnlyBuffer = new byte[keyLength]; + byte[] tempKey = new byte[keyLength]; + System.arraycopy(keyBuffer, 0, tempKey, 0, keyLength); + this.keyOnlyBuffer = ByteBuffer.wrap(tempKey); this.tagCompressionContext = tagCompressionContext; this.rowLength = rowLength; this.familyOffset = familyOffset; @@ -365,27 +408,32 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { this.valueOffset = valueOffset; this.tagsOffset = tagsOffset; this.tagsLength = tagsLength; - System.arraycopy(keyBuffer, 0, keyOnlyBuffer, 0, keyLength); if (tagCompressionContext != null) { - this.cloneTagsBuffer = new byte[tagsLength]; - System.arraycopy(tagsBuffer, 0, this.cloneTagsBuffer, 0, tagsLength); + byte[] tempTags = new byte[tagsLength]; + System.arraycopy(tagsBuffer, 0, tempTags, 0, tagsLength); + this.cloneTagsBuffer = ByteBuffer.wrap(tempTags); } setSequenceId(seqId); } + + @Override + public boolean hasArray() { + return false; + } @Override public byte[] getRowArray() { - return keyOnlyBuffer; + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override public byte[] getFamilyArray() { - return keyOnlyBuffer; + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override public byte[] getQualifierArray() { - return keyOnlyBuffer; + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override @@ -441,12 +489,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public byte[] getValueArray() { - return currentBuffer.array(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override public int getValueOffset() { - return currentBuffer.arrayOffset() + valueOffset; + return valueOffset; } @Override @@ -456,10 +504,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public byte[] getTagsArray() { - if (tagCompressionContext != null) { - return cloneTagsBuffer; - } - return currentBuffer.array(); + throw new UnsupportedOperationException("byte[] based API not supported"); } @Override @@ -467,7 +512,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { if (tagCompressionContext != null) { return 0; } - return currentBuffer.arrayOffset() + tagsOffset; + return tagsOffset; } @Override @@ -501,7 +546,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public String toString() { - return KeyValue.keyToString(this.keyOnlyBuffer, 0, KeyValueUtil.keyLength(this)) + "/vlen=" + return KeyValue.keyToString(this.keyOnlyBuffer.array(), 0, KeyValueUtil.keyLength(this)) + "/vlen=" + getValueLength() + "/seqid=" + seqId; } @@ -514,6 +559,72 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { public long heapSize() { return FIXED_OVERHEAD + rowLength + familyLength + qualifierLength + valueLength + tagsLength; } + + @Override + public ByteBuffer getRowBuffer() { + return keyOnlyBuffer; + } + + @Override + public ByteBuffer getFamilyBuffer() { + return keyOnlyBuffer; + } + + @Override + public ByteBuffer getQualifierBuffer() { + return keyOnlyBuffer; + } + + @Override + public ByteBuffer getValueBuffer() { + return currentBuffer; + } + + @Override + public ByteBuffer getTagsBuffer() { + if (tagCompressionContext != null) { + return cloneTagsBuffer; + } + return currentBuffer; + } + + @Override + public long oswrite(OutputStream out, boolean withTags) throws IOException { + int lenToWrite = KeyValueUtil.length(rowLength, familyLength, qualifierLength, valueLength, + tagsLength, withTags); + if (out instanceof ByteBufferOutputStream) { + // Write total length + ((ByteBufferOutputStream) out).writeInt(lenToWrite); + // write key length + ((ByteBufferOutputStream) out).writeInt(keyOnlyBuffer.capacity()); + // write value length + ((ByteBufferOutputStream) out).writeInt(valueLength); + } else { + StreamUtils.writeInt(out, lenToWrite); + StreamUtils.writeInt(out, keyOnlyBuffer.capacity()); + StreamUtils.writeInt(out, valueLength); + } + // Write key + assert keyOnlyBuffer.hasArray(); + out.write(keyOnlyBuffer.array()); + // Write value + ByteBufferUtils.writeToStream(out, this.currentBuffer, this.valueOffset, this.valueLength); + if (withTags) { + // 2 bytes tags length followed by tags bytes + // tags length is serialized with 2 bytes only(short way) even if the + // type is int. As this + // is non -ve numbers, we save the sign bit. See HBASE-11437 + out.write((byte) (0xff & (this.tagsLength >> 8))); + out.write((byte) (0xff & this.tagsLength)); + if (this.tagCompressionContext != null) { + assert cloneTagsBuffer.hasArray(); + out.write(cloneTagsBuffer.array()); + } else { + ByteBufferUtils.writeToStream(out, this.currentBuffer, this.tagsOffset, this.tagsLength); + } + } + return lenToWrite + Bytes.SIZEOF_INT; + } } protected abstract static class @@ -522,6 +633,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { protected HFileBlockDecodingContext decodingCtx; protected final KVComparator comparator; protected final SamePrefixComparator samePrefixComparator; + // This would directly refer to the HFileBlock protected ByteBuffer currentBuffer; protected STATE current = createSeekerState(); // always valid protected STATE previous = createSeekerState(); // may not be valid @@ -594,6 +706,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { @Override public ByteBuffer getKeyValueBuffer() { ByteBuffer kvBuffer = createKVBuffer(); + // Can be relative here because this is just a local copy kvBuffer.putInt(current.keyLength); kvBuffer.putInt(current.valueLength); kvBuffer.put(current.keyBuffer, 0, current.keyLength); @@ -620,8 +733,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { protected ByteBuffer createKVBuffer() { int kvBufSize = (int) KeyValue.getKeyValueDataStructureSize(current.keyLength, current.valueLength, current.tagsLength); - ByteBuffer kvBuffer = ByteBuffer.allocate(kvBufSize); - return kvBuffer; + return ByteBuffer.allocate(kvBufSize); } @Override @@ -854,14 +966,26 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { ByteBufferUtils.putCompressedInt(out, tagsLength); // There are some tags to be written if (tagsLength > 0) { + // Cells in write path is still byte[] based. + assert cell.hasArray(); TagCompressionContext tagCompressionContext = encodingCtx.getTagCompressionContext(); // When tag compression is enabled, tagCompressionContext will have a not null value. Write // the tags using Dictionary compression in such a case if (tagCompressionContext != null) { - tagCompressionContext - .compressTags(out, cell.getTagsArray(), cell.getTagsOffset(), tagsLength); + if (cell.hasArray()) { + tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(), + tagsLength); + } else { + tagCompressionContext.compressTags(out, cell.getTagsBuffer(), cell.getTagsOffset(), + tagsLength); + } } else { - out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength); + if (cell.hasArray()) { + out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength); + } else { + ByteBufferUtils.writeToStream(out, cell.getTagsBuffer(), cell.getTagsOffset(), + tagsLength); + } } } size += tagsLength + KeyValue.TAGS_LENGTH_SIZE; diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java index 6b87c77..7645840 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.io.WritableUtils; /** @@ -66,13 +67,13 @@ public class CopyKeyDataBlockEncoder extends BufferedDataBlockEncoder { } @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { + public ByteBuffer getFirstKeyInBlock(MultiByteBuffer block) { int keyLength = block.getInt(Bytes.SIZEOF_INT); - ByteBuffer dup = block.duplicate(); int pos = 3 * Bytes.SIZEOF_INT; - dup.position(pos); - dup.limit(pos + keyLength); - return dup.slice(); + ByteBuffer key = block.asSubBuffer(pos + keyLength).duplicate(); + key.position(pos); + key.limit(pos + keyLength); + return key.slice(); } @Override diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java index 872c22c..e767692 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Encoding of KeyValue. It aims to be fast and efficient using assumptions: @@ -90,7 +91,7 @@ public interface DataBlockEncoder { * @param block encoded block we want index, the position will not change * @return First key in block. */ - ByteBuffer getFirstKeyInBlock(ByteBuffer block); + ByteBuffer getFirstKeyInBlock(MultiByteBuffer block); /** * Create a HFileBlock seeker which find KeyValues within a block. diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java index 4182dc4..2c2ded6 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Compress using: @@ -227,7 +228,12 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { // put column family byte familyLength = cell.getFamilyLength(); out.write(familyLength); - out.write(cell.getFamilyArray(), cell.getFamilyOffset(), familyLength); + if (cell.hasArray()) { + out.write(cell.getFamilyArray(), cell.getFamilyOffset(), familyLength); + } else { + ByteBufferUtils.writeToStream(out, cell.getFamilyBuffer(), cell.getFamilyOffset(), + familyLength); + } } else { // Finding common prefix int preKeyLength = KeyValueUtil.keyLength(prevCell); @@ -280,7 +286,12 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { // Previous and current rows are different. Copy the differing part of // the row, skip the column family, and copy the qualifier. CellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out); - out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + if (cell.hasArray()) { + out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + } else { + ByteBufferUtils.writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset(), + cell.getQualifierLength()); + } } else { // The common part includes the whole row. As the column family is the // same across the whole file, it will automatically be included in the @@ -288,8 +299,13 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { // What we write here is the non common part of the qualifier int commonQualPrefix = commonPrefix - (rLen + KeyValue.ROW_LENGTH_SIZE) - (cell.getFamilyLength() + KeyValue.FAMILY_LENGTH_SIZE); - out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonQualPrefix, - cell.getQualifierLength() - commonQualPrefix); + if (cell.hasArray()) { + out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonQualPrefix, + cell.getQualifierLength() - commonQualPrefix); + } else { + ByteBufferUtils.writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset() + + commonQualPrefix, cell.getQualifierLength() - commonQualPrefix); + } } if ((flag & FLAG_TIMESTAMP_IS_DIFF) == 0) { ByteBufferUtils.putLong(out, timestamp, timestampFitsInBytes); @@ -300,20 +316,24 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { if ((flag & FLAG_SAME_TYPE) == 0) { out.write(cell.getTypeByte()); } - out.write(cell.getValueArray(), cell.getValueOffset(), vLength); + if (cell.hasArray()) { + out.write(cell.getValueArray(), cell.getValueOffset(), vLength); + } else { + ByteBufferUtils.writeToStream(out, cell.getValueBuffer(), cell.getValueOffset(), vLength); + } return kLength + vLength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; } @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { + public ByteBuffer getFirstKeyInBlock(MultiByteBuffer block) { block.mark(); block.position(Bytes.SIZEOF_INT); byte familyLength = block.get(); - ByteBufferUtils.skip(block, familyLength); + block.skip(familyLength); byte flag = block.get(); - int keyLength = ByteBufferUtils.readCompressedInt(block); - ByteBufferUtils.readCompressedInt(block); // valueLength - ByteBufferUtils.readCompressedInt(block); // commonLength + int keyLength = MultiByteBuffer.readCompressedInt(block); + MultiByteBuffer.readCompressedInt(block); // valueLength + MultiByteBuffer.readCompressedInt(block); // commonLength ByteBuffer result = ByteBuffer.allocate(keyLength); // copy row @@ -341,7 +361,7 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder { // copy the timestamp and type int timestampFitInBytes = ((flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH) + 1; - long timestamp = ByteBufferUtils.readLong(block, timestampFitInBytes); + long timestamp = MultiByteBuffer.readLong(block, timestampFitInBytes); if ((flag & FLAG_TIMESTAMP_SIGN) != 0) { timestamp = -timestamp; } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java index a6f43d0..e3e812d 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Encoder similar to {@link DiffKeyDeltaEncoder} but supposedly faster. @@ -262,7 +263,12 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { ByteBufferUtils.putCompressedInt(out, 0); CellUtil.writeFlatKey(cell, out); // Write the value part - out.write(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); + if (cell.hasArray()) { + out.write(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); + } else { + ByteBufferUtils.writeToStream(out, cell.getValueBuffer(), cell.getValueOffset(), + cell.getValueLength()); + } } else { int preKeyLength = KeyValueUtil.keyLength(prevCell); int preValLength = prevCell.getValueLength(); @@ -287,9 +293,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { // Check if current and previous values are the same. Compare value // length first as an optimization. - if (vLength == preValLength - && Bytes.equals(cell.getValueArray(), cell.getValueOffset(), vLength, - prevCell.getValueArray(), prevCell.getValueOffset(), preValLength)) { + if (vLength == preValLength && CellUtil.matchingValue(cell, prevCell)) { flag |= FLAG_SAME_VALUE; } @@ -306,7 +310,12 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { // Previous and current rows are different. Copy the differing part of // the row, skip the column family, and copy the qualifier. CellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out); - out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + if (cell.hasArray()) { + out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + } else { + ByteBufferUtils.writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset(), + cell.getQualifierLength()); + } } else { // The common part includes the whole row. As the column family is the // same across the whole file, it will automatically be included in the @@ -314,8 +323,13 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { // What we write here is the non common part of the qualifier int commonQualPrefix = commonPrefix - (rLen + KeyValue.ROW_LENGTH_SIZE) - (cell.getFamilyLength() + KeyValue.FAMILY_LENGTH_SIZE); - out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonQualPrefix, - cell.getQualifierLength() - commonQualPrefix); + if (cell.hasArray()) { + out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonQualPrefix, + cell.getQualifierLength() - commonQualPrefix); + } else { + ByteBufferUtils.writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset() + + commonQualPrefix, cell.getQualifierLength() - commonQualPrefix); + } } // Write non common ts part out.write(curTsBuf, commonTimestampPrefix, KeyValue.TIMESTAMP_SIZE - commonTimestampPrefix); @@ -327,7 +341,11 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { // Write the value if it is not the same as before. if ((flag & FLAG_SAME_VALUE) == 0) { - out.write(cell.getValueArray(), cell.getValueOffset(), vLength); + if (cell.hasArray()) { + out.write(cell.getValueArray(), cell.getValueOffset(), vLength); + } else { + ByteBufferUtils.writeToStream(out, cell.getValueBuffer(), cell.getValueOffset(), vLength); + } } } return kLength + vLength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; @@ -354,18 +372,16 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder { } @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { + public ByteBuffer getFirstKeyInBlock(MultiByteBuffer block) { block.mark(); block.position(Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE); - int keyLength = ByteBufferUtils.readCompressedInt(block); - ByteBufferUtils.readCompressedInt(block); // valueLength - ByteBufferUtils.readCompressedInt(block); // commonLength - int pos = block.position(); + int keyLength = MultiByteBuffer.readCompressedInt(block); + MultiByteBuffer.readCompressedInt(block); // valueLength + MultiByteBuffer.readCompressedInt(block); // commonLength + ByteBuffer key = block.asSubBuffer(keyLength).duplicate(); block.reset(); - ByteBuffer dup = block.duplicate(); - dup.position(pos); - dup.limit(pos + keyLength); - return dup.slice(); + key.limit(key.position() + keyLength); + return key.slice(); } @Override diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java index 37001cc..6208ef0 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDecodingContext.java @@ -17,10 +17,10 @@ package org.apache.hadoop.hbase.io.encoding; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * A decoding context that is created by a reader's encoder, and is shared @@ -46,8 +46,8 @@ public interface HFileBlockDecodingContext { void prepareDecoding( int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, - ByteBuffer blockBufferWithoutHeader, - ByteBuffer onDiskBlock + MultiByteBuffer blockBufferWithoutHeader, + MultiByteBuffer onDiskBlock ) throws IOException; /** diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java index 78bb0d6..28b9cb9 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultDecodingContext.java @@ -19,11 +19,10 @@ package org.apache.hadoop.hbase.io.encoding; import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.ByteBuffer; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.io.ByteBufferInputStream; +import org.apache.hadoop.hbase.io.MultiByteBufferInputStream; import org.apache.hadoop.hbase.io.TagCompressionContext; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.crypto.Cipher; @@ -31,6 +30,7 @@ import org.apache.hadoop.hbase.io.crypto.Decryptor; import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * A default implementation of {@link HFileBlockDecodingContext}. It assumes the @@ -51,8 +51,9 @@ public class HFileBlockDefaultDecodingContext implements @Override public void prepareDecoding(int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, - ByteBuffer blockBufferWithoutHeader, ByteBuffer onDiskBlock) throws IOException { - InputStream in = new DataInputStream(new ByteBufferInputStream(onDiskBlock)); + MultiByteBuffer blockBufferWithoutHeader, MultiByteBuffer onDiskBlock) + throws IOException { + InputStream in = new DataInputStream(new MultiByteBufferInputStream(onDiskBlock)); Encryption.Context cryptoContext = fileContext.getEncryptionContext(); if (cryptoContext != Encryption.Context.NONE) { diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java index 0286eca..478d794 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Compress key by storing size of common prefix with previous KeyValue @@ -68,7 +69,12 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { writeKeyExcludingCommon(cell, common, out); } // Write the value part - out.write(cell.getValueArray(), cell.getValueOffset(), vlength); + if (cell.hasArray()) { + out.write(cell.getValueArray(), cell.getValueOffset(), vlength); + } else { + ByteBufferUtils + .copyBufferToStream(out, cell.getValueBuffer(), cell.getValueOffset(), vlength); + } int size = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; size += afterEncodingKeyValue(cell, out, encodingContext); state.prevCell = cell; @@ -84,8 +90,14 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { CellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out); byte fLen = cell.getFamilyLength(); out.writeByte(fLen); - out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen); - out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + if (cell.hasArray()) { + out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen); + out.write(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + } else { + ByteBufferUtils.writeToStream(out, cell.getFamilyBuffer(), cell.getFamilyOffset(), fLen); + ByteBufferUtils.writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset(), + cell.getQualifierLength()); + } out.writeLong(cell.getTimestamp()); out.writeByte(cell.getTypeByte()); } else { @@ -97,8 +109,13 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { int commonQualPrefix = Math.min(commonPrefix, qLen); int qualPartLenToWrite = qLen - commonQualPrefix; if (qualPartLenToWrite > 0) { - out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonQualPrefix, - qualPartLenToWrite); + if (cell.hasArray()) { + out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonQualPrefix, + qualPartLenToWrite); + } else { + ByteBufferUtils.writeToStream(out, cell.getQualifierBuffer(), cell.getQualifierOffset() + + commonQualPrefix, qualPartLenToWrite); + } } commonPrefix -= commonQualPrefix; // Common part in TS also? @@ -172,22 +189,20 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder { } @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { + public ByteBuffer getFirstKeyInBlock(MultiByteBuffer block) { block.mark(); block.position(Bytes.SIZEOF_INT); - int keyLength = ByteBufferUtils.readCompressedInt(block); - ByteBufferUtils.readCompressedInt(block); - int commonLength = ByteBufferUtils.readCompressedInt(block); + int keyLength = MultiByteBuffer.readCompressedInt(block); + MultiByteBuffer.readCompressedInt(block); + int commonLength = MultiByteBuffer.readCompressedInt(block); if (commonLength != 0) { throw new AssertionError("Nonzero common length in the first key in " + "block: " + commonLength); } - int pos = block.position(); + ByteBuffer key = block.asSubBuffer(keyLength).duplicate(); block.reset(); - ByteBuffer dup = block.duplicate(); - dup.position(pos); - dup.limit(pos + keyLength); - return dup.slice(); + key.limit(key.position() + keyLength); + return key.slice(); } @Override diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java index 0db584e..b679f6d 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Various types of HFile blocks. Ordinal values of these enum constants must not be relied upon. @@ -131,7 +132,7 @@ public enum BlockType { out.write(magic); } - public void write(ByteBuffer buf) { + public void write(MultiByteBuffer buf) { buf.put(magic); } @@ -161,7 +162,7 @@ public enum BlockType { return parse(buf, 0, buf.length); } - public static BlockType read(ByteBuffer buf) throws IOException { + public static BlockType read(MultiByteBuffer buf) throws IOException { byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), MAGIC_LENGTH)]; buf.get(magicBuf); BlockType blockType = parse(magicBuf, 0, magicBuf.length); diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java index 5f43444..c6bbde7 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java @@ -58,6 +58,8 @@ public class HFileContext implements HeapSize, Cloneable { private Encryption.Context cryptoContext = Encryption.Context.NONE; private long fileCreateTime; + private String hfileName; + //Empty constructor. Go with setters public HFileContext() { } @@ -78,12 +80,13 @@ public class HFileContext implements HeapSize, Cloneable { this.encoding = context.encoding; this.cryptoContext = context.cryptoContext; this.fileCreateTime = context.fileCreateTime; + this.hfileName = context.hfileName; } public HFileContext(boolean useHBaseChecksum, boolean includesMvcc, boolean includesTags, Compression.Algorithm compressAlgo, boolean compressTags, ChecksumType checksumType, int bytesPerChecksum, int blockSize, DataBlockEncoding encoding, - Encryption.Context cryptoContext, long fileCreateTime) { + Encryption.Context cryptoContext, long fileCreateTime, String name) { this.usesHBaseChecksum = useHBaseChecksum; this.includesMvcc = includesMvcc; this.includesTags = includesTags; @@ -97,6 +100,7 @@ public class HFileContext implements HeapSize, Cloneable { } this.cryptoContext = cryptoContext; this.fileCreateTime = fileCreateTime; + this.hfileName = name; } /** @@ -188,6 +192,10 @@ public class HFileContext implements HeapSize, Cloneable { this.cryptoContext = cryptoContext; } + public String getHFileName() { + return this.hfileName; + } + /** * HeapSize implementation * NOTE : The heapsize should be altered as and when new state variable are added @@ -196,12 +204,15 @@ public class HFileContext implements HeapSize, Cloneable { @Override public long heapSize() { long size = ClassSize.align(ClassSize.OBJECT + - // Algorithm reference, encodingon, checksumtype, Encryption.Context reference - 4 * ClassSize.REFERENCE + + // Algorithm reference, encodingon, checksumtype, Encryption.Context reference, hfileName + 5 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT + // usesHBaseChecksum, includesMvcc, includesTags and compressTags 4 * Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_LONG); + if (this.hfileName != null) { + size += ClassSize.align(1 * ClassSize.STRING + (2 * this.hfileName.length())); + } return size; } @@ -228,6 +239,10 @@ public class HFileContext implements HeapSize, Cloneable { sb.append(" compressAlgo="); sb.append(compressAlgo); sb.append(" compressTags="); sb.append(compressTags); sb.append(" cryptoContext=[ "); sb.append(cryptoContext); sb.append(" ]"); + if (hfileName != null) { + sb.append(" HFile name="); + sb.append(hfileName); + } sb.append(" ]"); return sb.toString(); } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java index 0d1e6ef..554b8ec 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java @@ -53,6 +53,8 @@ public class HFileContextBuilder { /** Crypto context */ private Encryption.Context cryptoContext = Encryption.Context.NONE; private long fileCreateTime = 0; + // Path name + private String hfileName; public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) { this.usesHBaseChecksum = useHBaseCheckSum; @@ -108,10 +110,14 @@ public class HFileContextBuilder { this.fileCreateTime = fileCreateTime; return this; } + public HFileContextBuilder withPathName(String name) { + this.hfileName = name; + return this; + } public HFileContext build() { return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compression, compressTags, checksumType, bytesPerChecksum, blocksize, encoding, cryptoContext, - fileCreateTime); + fileCreateTime, hfileName); } } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java index 314ed2b..24e49b3 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java @@ -118,6 +118,41 @@ public class StreamUtils { return result; } + public static Pair readRawVarint32(ByteBuffer input, int offset) throws IOException { + int newOffset = offset; + byte tmp = input.get(newOffset++); + if (tmp >= 0) { + return new Pair((int) tmp, newOffset - offset); + } + int result = tmp & 0x7f; + if ((tmp = input.get(newOffset++)) >= 0) { + result |= tmp << 7; + } else { + result |= (tmp & 0x7f) << 7; + if ((tmp = input.get(newOffset++)) >= 0) { + result |= tmp << 14; + } else { + result |= (tmp & 0x7f) << 14; + if ((tmp = input.get(newOffset++)) >= 0) { + result |= tmp << 21; + } else { + result |= (tmp & 0x7f) << 21; + result |= (tmp = input.get(newOffset++)) << 28; + if (tmp < 0) { + // Discard upper 32 bits. + for (int i = 0; i < 5; i++) { + if (input.get(newOffset++) >= 0) { + return new Pair(result, newOffset - offset); + } + } + throw new IOException("Malformed varint"); + } + } + } + } + return new Pair(result, newOffset - offset); + } + /** * Reads a varInt value stored in an array. * diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java index 3d6d260..269b453 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java @@ -195,4 +195,54 @@ public final class ByteBufferArray { } assert srcIndex == len; } + + // TODO better name + // TODO : Add test for this + public MultiByteBuffer subArray(long offset, int len) { + assert len >= 0; + long end = offset + len; + int startBuffer = (int) (offset / bufferSize), startBufferOffset = (int) (offset % bufferSize); + int endBuffer = (int) (end / bufferSize), endBufferOffset = (int) (end % bufferSize); + assert startBuffer >= 0 && startBuffer < bufferCount; + assert endBuffer >= 0 && endBuffer < bufferCount + || (endBuffer == bufferCount && endBufferOffset == 0); + if (startBuffer >= locks.length || startBuffer < 0) { + String msg = "Failed subArray, start=" + offset + ",startBuffer=" + startBuffer + + ",bufferSize=" + bufferSize; + LOG.error(msg); + throw new RuntimeException(msg); + } + int srcIndex = 0, cnt = -1; + ByteBuffer[] mbb = new ByteBuffer[endBuffer - startBuffer + 1]; + for (int i = startBuffer,j=0; i <= endBuffer; ++i,j++) { + Lock lock = locks[i]; + lock.lock(); + try { + ByteBuffer bb = buffers[i]; + if (i == startBuffer) { + cnt = bufferSize - startBufferOffset; + if (cnt > len) cnt = len; + ByteBuffer dup = bb.duplicate(); + dup.limit(startBufferOffset + cnt).position(startBufferOffset); + mbb[j] = dup.slice(); + } else if (i == endBuffer) { + cnt = endBufferOffset; + ByteBuffer dup = bb.duplicate(); + dup.position(0).limit(cnt); + mbb[j] = dup.slice(); + } else { + cnt = bufferSize ; + ByteBuffer dup = bb.duplicate(); + dup.position(0).limit(cnt); + mbb[j] = dup.slice(); + } + srcIndex += cnt; + } finally { + lock.unlock(); + } + } + assert srcIndex == len; + MultiByteBuffer bb = new MultiByteBuffer(mbb); + return bb; + } } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java index 6f348bc..c6ee88b 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java @@ -21,13 +21,24 @@ import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.nio.ByteBuffer; +import java.security.AccessController; +import java.security.PrivilegedAction; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.io.ByteBufferOutputStream; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.WritableUtils; +import sun.misc.Unsafe; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.Charset; + +import sun.nio.ch.DirectBuffer; +import java.lang.reflect.Field; + /** * Utility functions for working with byte buffers, such as reading/writing * variable-length long numbers. @@ -36,10 +47,13 @@ import org.apache.hadoop.io.WritableUtils; @InterfaceStability.Evolving public final class ByteBufferUtils { + private static final boolean UNSAFE_AVAIL = UnsafeUtil.isAvailable(); // "Compressed integer" serialization helper constants. - private final static int VALUE_MASK = 0x7f; - private final static int NEXT_BIT_SHIFT = 7; - private final static int NEXT_BIT_MASK = 1 << 7; + private static final String UTF8_ENCODING = "UTF-8"; + public final static int VALUE_MASK = 0x7f; + public final static int NEXT_BIT_SHIFT = 7; + public final static int NEXT_BIT_MASK = 1 << 7; + private static final Charset UTF8_CHARSET = Charset.forName(UTF8_ENCODING); private ByteBufferUtils() { } @@ -76,7 +90,7 @@ public final class ByteBufferUtils { out.put((byte) ((i & mask) >> shiftbits)); } } - + /** * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a * {@link ByteBuffer}. @@ -96,7 +110,6 @@ public final class ByteBufferUtils { return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i); } - /** * Put in buffer integer using 7 bit encoding. For each written byte: * 7 bits are used to store value @@ -146,7 +159,7 @@ public final class ByteBufferUtils { copyBufferToStream(out, in, in.position(), length); skip(in, length); } - + /** * Copy data from a buffer to an output stream. Does not update the position * in the buffer. @@ -159,8 +172,7 @@ public final class ByteBufferUtils { public static void copyBufferToStream(OutputStream out, ByteBuffer in, int offset, int length) throws IOException { if (in.hasArray()) { - out.write(in.array(), in.arrayOffset() + offset, - length); + out.write(in.array(), in.arrayOffset() + offset, length); } else { for (int i = 0; i < length; ++i) { out.write(in.get(offset + i)); @@ -236,6 +248,16 @@ public final class ByteBufferUtils { return 4; } + public static ByteBuffer copyOfRange(ByteBuffer original, int from, int to) { + int newLength = to - from; + if (newLength < 0) + throw new IllegalArgumentException(from + " > " + to); + ByteBuffer copy = ByteBuffer.allocate(newLength); + ByteBufferUtils.copyFromBufferToBuffer(copy, original, from, 0, + Math.min(original.capacity() - from, newLength)); + return copy; + } + /** * Read integer from stream coded in 7 bits and increment position. * @return the integer that has been read @@ -269,7 +291,7 @@ public final class ByteBufferUtils { } return b & VALUE_MASK; } - + /** * Read long which was written to fitInBytes bytes and increment position. * @param fitInBytes In how many bytes given long is stored. @@ -297,7 +319,7 @@ public final class ByteBufferUtils { } return tmpLength; } - + /** * Copy the given number of bytes from the given stream and put it at the * current position of the given buffer, updating the position in the buffer. @@ -319,7 +341,7 @@ public final class ByteBufferUtils { } /** - * Copy from the InputStream to a new heap ByteBuffer until the InputStream is exhausted. + * Copy from the InputStream to a new heap ByteRange until the InputStream is exhausted. */ public static ByteBuffer drainInputStreamToBuffer(InputStream is) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(4096); @@ -342,15 +364,18 @@ public final class ByteBufferUtils { public static void copyFromBufferToBuffer(ByteBuffer out, ByteBuffer in, int sourceOffset, int length) { if (in.hasArray() && out.hasArray()) { - System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), - out.array(), out.position() + - out.arrayOffset(), length); - skip(out, length); + System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.position() + + out.arrayOffset(), length); + } else if (UNSAFE_AVAIL) { + UnsafeUtil.copy(in, sourceOffset, out, out.position(), length); } else { + // TODO buf.put(buf) will be better.. Deal with pos then. + int destOffset = out.position(); for (int i = 0; i < length; ++i) { - out.put(in.get(sourceOffset + i)); + putByte(out, destOffset + i, getByte(in, sourceOffset + i)); } } + skip(out, length); } /** @@ -362,37 +387,56 @@ public final class ByteBufferUtils { * @param destinationOffset * @param length */ - public static void copyFromBufferToBuffer(ByteBuffer out, ByteBuffer in, int sourceOffset, + public static int copyFromBufferToBuffer(ByteBuffer out, ByteBuffer in, int sourceOffset, int destinationOffset, int length) { if (in.hasArray() && out.hasArray()) { System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.arrayOffset() + destinationOffset, length); + } else if (UNSAFE_AVAIL) { + UnsafeUtil.copy(in, sourceOffset, out, destinationOffset, length); } else { + // TODO buf.put(buf) will be better.. Deal with pos then. for (int i = 0; i < length; ++i) { - out.put((destinationOffset + i), in.get(sourceOffset + i)); + putByte(out, destinationOffset + i, getByte(in, sourceOffset + i)); } } + return destinationOffset + length; } - /** - * Find length of common prefix of two parts in the buffer - * @param buffer Where parts are located. - * @param offsetLeft Offset of the first part. - * @param offsetRight Offset of the second part. - * @param limit Maximal length of common prefix. - * @return Length of prefix. - */ - public static int findCommonPrefix(ByteBuffer buffer, int offsetLeft, - int offsetRight, int limit) { - int prefix = 0; - - for (; prefix < limit; ++prefix) { - if (buffer.get(offsetLeft + prefix) != buffer.get(offsetRight + prefix)) { - break; + public static int copyFromBufferToByteArray(byte[] out, ByteBuffer in, + int sourceOffset, int destinationOffset, int length) { + if (in.hasArray()) { + System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out, destinationOffset, length); + } else if (UNSAFE_AVAIL) { + UnsafeUtil.copy(in, sourceOffset, out, destinationOffset, length); + } else { + for (int i = 0; i < length; i++) { + out[destinationOffset + i] = in.get(sourceOffset + i); } } + return destinationOffset + length; + } - return prefix; + /** + * Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes + * to buffer's current position. This also advances the position in the buffer by 'length' + * @param out + * @param in + * @param inOffset + * @param length + */ + public static void copyFromArrayToBuffer(ByteBuffer out, byte[] in, int inOffset, int length) { + if (out.hasArray()) { + System.arraycopy(in, inOffset, out.array(), out.arrayOffset() + out.position(), length); + // Move the position in out by length + out.position(out.position() + length); + } else if (UNSAFE_AVAIL) { + UnsafeUtil.copy(in, inOffset, out, out.position(), length); + // Move the position in out by length + out.position(out.position() + length); + } else { + out.put(in, inOffset, length); + } } /** @@ -417,6 +461,20 @@ public final class ByteBufferUtils { return result; } + + public static int findCommonPrefix( + ByteBuffer left, int leftOffset, int leftLength, + ByteBuffer right, int rightOffset, int rightLength) { + int length = Math.min(leftLength, rightLength); + int result = 0; + + while (result < length && + left.get(leftOffset + result) == right.get(rightOffset + result)) { + result++; + } + + return result; + } /** * Check whether two parts in the same buffer are equal. @@ -447,7 +505,7 @@ public final class ByteBufferUtils { } return true; } - + /** * Increment position in buffer. * @param buffer In this buffer. @@ -456,10 +514,11 @@ public final class ByteBufferUtils { public static void skip(ByteBuffer buffer, int length) { buffer.position(buffer.position() + length); } - + public static void extendLimit(ByteBuffer buffer, int numBytes) { buffer.limit(buffer.limit() + numBytes); } + /** * Copy the bytes from position to limit into a new byte[] of the exact length and sets the @@ -477,35 +536,665 @@ public final class ByteBufferUtils { return output; } + public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) { + if (buf1.hasArray() && buf2.hasArray()) { + return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, l1, buf2.array(), + buf2.arrayOffset() + o2, l2); + } + if (UNSAFE_AVAIL) { + return UnsafeUtil.compareTo(buf1, o1, l1, buf2, o2, l2); + } + int end1 = o1 + l1; + int end2 = o2 + l2; + for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) { + int a = buf1.get(i) & 0xFF; + int b = buf2.get(j) & 0xFF; + if (a != b) { + return a - b; + } + } + return l1 - l2; + } + + public static String toString(final ByteBuffer b) { + return toString(b, 0, b.capacity()); + } + /** - * Copy the given number of bytes from specified offset into a new byte[] - * @param buffer - * @param offset - * @param length - * @return a new byte[] containing the bytes in the specified range + * This method will convert utf8 encoded bytes into a string. If + * the given byte array is null, this method will return null. + * + * @param b Presumed UTF-8 encoded byte array. + * @param off offset into array + * @param len length of utf-8 sequence + * @return String made from b or null */ - public static byte[] toBytes(ByteBuffer buffer, int offset, int length) { - byte[] output = new byte[length]; - for (int i = 0; i < length; i++) { - output[i] = buffer.get(offset + i); + public static String toString(final ByteBuffer b, int off, int len) { + if (b == null) { + return null; + } + if (len == 0) { + return ""; + } + byte[] bytes = new byte[len]; + ByteBufferUtils.copyFromBufferToByteArray(bytes, b, off, 0, len); + return new String(bytes, 0, len, UTF8_CHARSET); + } + + + public static int putShort(ByteBuffer buffer, int offset, short val) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.putShortUnsafe(buffer, offset, val); + } else { + buffer.put(offset + 1, (byte) val); + val >>= 8; + buffer.put(offset, (byte) val); + return offset + Bytes.SIZEOF_SHORT; + } + } + + public static int putShort(ByteBuffer buffer, int offset, int val) { + buffer.put(offset + 1, (byte) val); + val >>= 8; + buffer.put(offset, (byte) val); + return offset + Bytes.SIZEOF_SHORT; + } + + public static int putInt(ByteBuffer buffer, int offset, int val) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.putIntUnsafe(buffer, offset, val); + } else { + for(int i= offset + 3; i > offset; i--) { + buffer.put(i, (byte)val); + val >>>= 8; + } + buffer.put(offset, (byte)val); + return offset + Bytes.SIZEOF_INT; } - return output; } - public static int compareTo(ByteBuffer buf1, int o1, int len1, ByteBuffer buf2, int o2, int len2) { - if (buf1.hasArray() && buf2.hasArray()) { - return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(), - buf2.arrayOffset() + o2, len2); + public static int putFloat(ByteBuffer buffer, int offset, float f) { + return putInt(buffer, offset, Float.floatToRawIntBits(f)); + } + + public static int putDouble(ByteBuffer buffer, int offset, double d) { + return putLong(buffer, offset, Double.doubleToLongBits(d)); + } + + public static int putByte(ByteBuffer buffer, int offset, byte b) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.putByte(buffer, offset, b); + } else { + buffer.put(offset, b); + return offset + 1; } - int end1 = o1 + len1; - int end2 = o2 + len2; + } + public static int putLong(ByteBuffer buffer, int offset, long val) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.putLongUnsafe(buffer, offset, val); + } else { + for(int i = offset + 7; i > offset; i--) { + buffer.put(i, (byte)val); + val >>>= 8; + } + buffer.put(offset, (byte)val); + return offset + Bytes.SIZEOF_LONG; + } + } + + public static int putAsShort(ByteBuffer buffer, int offset, int val) { + putByte(buffer, offset + 1, (byte) val); + val >>= 8; + putByte(buffer, offset, (byte) val); + return offset + Bytes.SIZEOF_SHORT; + } + + // Need test case for this + public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) { + if (buf1.hasArray()) { + return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, l1, buf2, o2, l2); + } + if (UNSAFE_AVAIL) { + return UnsafeUtil.compareTo(buf1, o1, l1, buf2, o2, l2); + } + // Bring WritableComparator code local + int end1 = o1 + l1; + int end2 = o2 + l2; for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) { - int a = buf1.get(i) & 0xFF; - int b = buf2.get(j) & 0xFF; + int a = (buf1.get(i) & 0xff); + int b = (buf2[j] & 0xff); if (a != b) { return a - b; } } - return len1 - len2; + return l1 - l2; + } + + public static int hashCode(ByteBuffer buffer, int offset, int length) { + int hash = 1; + for (int i = offset; i < offset + length; i++) + hash = (31 * hash) + (int) buffer.get(i); + return hash; + } + + public static int compareTo(byte[] buf1, int o1, int l1, ByteBuffer right, int o2, int l2) { + int c = compareTo(right, o2, l2, buf1, o1, l1); + return -(c); + } + + public static boolean equals(final ByteBuffer left, int leftOffset, int leftLen, + final byte[] right, int rightOffset, int rightLen) { + // different lengths fast check + if (leftLen != rightLen) { + return false; + } + if (leftLen == 0) { + return true; + } + // Since we're often comparing adjacent sorted data, + // it's usual to have equal arrays except for the very last byte + // so check that first + // The getOffset() should not be used here + if (getByte(left, leftOffset + leftLen - 1) != right[rightOffset + rightLen - 1]) { + return false; + } + return compareTo(left, leftOffset, leftLen, right, rightOffset, rightLen) == 0; + } + + public static boolean equals(final ByteBuffer left, int leftOffset, int leftLen, final ByteBuffer right, + int rightOffset, int rightLen) { + // different lengths fast check + if (leftLen != rightLen) { + return false; + } + if (leftLen == 0) { + return true; + } + // Since we're often comparing adjacent sorted data, + // it's usual to have equal arrays except for the very last byte + // so check that first + if (getByte(left, leftOffset + leftLen - 1) != getByte(right, rightOffset + rightLen - 1)) { + return false; + } + return compareTo(left, leftOffset, leftLen, right, rightOffset, rightLen) == 0; + } + + public static boolean equals(final byte[] left, int leftOffset, int leftLen, final ByteBuffer right, + int rightOffset, int rightLen) { + // different lengths fast check + if (leftLen != rightLen) { + return false; + } + if (leftLen == 0) { + return true; + } + // Since we're often comparing adjacent sorted data, + // it's usual to have equal arrays except for the very last byte + // so check that first + if (left[leftOffset + leftLen - 1] != getByte(right, rightOffset + rightLen - 1)) { + return false; + } + return compareTo(left, leftOffset, leftLen, right, rightOffset, rightLen) == 0; + } + + public static int getInt(ByteBuffer buffer, int offset) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.toIntUnsafe(buffer, offset); + } else { + int n = 0; + for (int i = offset; i < (offset + Bytes.SIZEOF_INT); i++) { + n <<= 8; + n ^= getByte(buffer, i) & 0xFF; + } + return n; + } + } + + public static int getAsInt(ByteBuffer buffer, int offset, final int length) { + if (length == Bytes.SIZEOF_INT) { + return getInt(buffer, offset); + } + int n = 0; + for (int i = offset; i < (offset + length); i++) { + n <<= 8; + n ^= getByte(buffer, i) & 0xFF; + } + return n; + } + + public static String toStringBinary(final ByteBuffer b, int off, int len) { + StringBuilder result = new StringBuilder(); + // Just in case we are passed a 'len' that is > buffer length... + if (off >= b.capacity()) + return result.toString(); + if (off + len > b.capacity()) + len = b.capacity() - off; + for (int i = off; i < off + len; ++i) { + int ch = b.get(i) & 0xFF; + if ((ch >= '0' && ch <= '9') || (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z') + || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0) { + result.append((char) ch); + } else { + result.append(String.format("\\x%02X", ch)); + } + } + return result.toString(); + } + + public static String toStringBinary(final ByteBuffer b) { + return toStringBinary(b, 0, b.capacity()); + } + + public static byte getByte(ByteBuffer buffer, int offset) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.get(buffer, offset); + } else { + return buffer.get(offset); + } + } + + public static long getLong(ByteBuffer buffer, int offset) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.toLongUnsafe(buffer, offset); + } else { + long l = 0; + for (int i = offset; i < offset + Bytes.SIZEOF_LONG; i++) { + l <<= 8; + l ^= buffer.get(i) & 0xFF; + } + return l; + } + } + + public static long getAsLong(ByteBuffer buffer, int offset, final int length) { + if (length == Bytes.SIZEOF_LONG) { + return getLong(buffer, offset); + } + long l = 0; + for (int i = offset; i < offset + length; i++) { + l <<= 8; + l ^= buffer.get(i) & 0xFF; + } + return l; } + + public static short getShort(ByteBuffer buffer, int offset) { + if (UNSAFE_AVAIL) { + return UnsafeUtil.toShortUnsafe(buffer, offset); + } else { + short n = 0; + n ^= buffer.get(offset) & 0xFF; + n <<= 8; + n ^= buffer.get(offset + 1) & 0xFF; + return n; + } + } + + public static void writeToStream(OutputStream out, ByteBuffer buf, int offset, int length) + throws IOException { + if (out instanceof ByteBufferOutputStream) { + ((ByteBufferOutputStream) out).write(buf, offset, length); + } else { + if (buf.hasArray()) { + out.write(buf.array(), buf.arrayOffset() + offset, length); + } else { + // Here we have to create temp array and do copy!! + byte[] b = new byte[length]; + ByteBufferUtils.copyFromBufferToByteArray(b, buf, offset, 0, length); + out.write(b); + } + } + } + + static class UnsafeUtil { + static final Unsafe theUnsafe; + + /** The offset to the first element in a byte array. */ + static long BYTE_ARRAY_BASE_OFFSET; + + static { + theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction() { + @Override + public Object run() { + try { + Field f = Unsafe.class.getDeclaredField("theUnsafe"); + f.setAccessible(true); + return f.get(null); + } catch (Exception e) { + return null; + } + } + }); + if (theUnsafe != null) + BYTE_ARRAY_BASE_OFFSET = theUnsafe.arrayBaseOffset(byte[].class); + } + + static final boolean littleEndian = ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN); + + static boolean isAvailable() { + return theUnsafe != null; + } + + static boolean lessThanUnsignedLong(long x1, long x2) { + if (littleEndian) { + x1 = Long.reverseBytes(x1); + x2 = Long.reverseBytes(x2); + } + return (x1 + Long.MIN_VALUE) < (x2 + Long.MIN_VALUE); + } + + static boolean lessThanUnsignedInt(int x1, int x2) { + if (littleEndian) { + x1 = Integer.reverseBytes(x1); + x2 = Integer.reverseBytes(x2); + } + return (x1 & 0xffffffffL) < (x2 & 0xffffffffL); + } + + static boolean lessThanUnsignedShort(short x1, short x2) { + if (littleEndian) { + x1 = Short.reverseBytes(x1); + x2 = Short.reverseBytes(x2); + } + return (x1 & 0xffff) < (x2 & 0xffff); + } + + static long getLong(ByteBuffer buf, int offset) { + if (buf.isDirect()) { + return theUnsafe.getLong(((DirectBuffer) buf).address() + offset); + } else { + assert buf.hasArray(); + return theUnsafe.getLong(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset); + } + } + + static int putShort(ByteBuffer buf, int offset, short val) { + if (buf.isDirect()) { + theUnsafe.putShort(((DirectBuffer) buf).address() + offset, val); + } else { + assert buf.hasArray(); + theUnsafe.putShort(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, val); + } + return offset + Bytes.SIZEOF_SHORT; + } + + static int putInt(ByteBuffer buf, int offset, int val) { + if (buf.isDirect()) { + theUnsafe.putInt(((DirectBuffer) buf).address() + offset, val); + } else { + assert buf.hasArray(); + theUnsafe.putInt(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, val); + } + return offset + Bytes.SIZEOF_INT; + } + + static int putLong(ByteBuffer buf, int offset, long val) { + if (buf.isDirect()) { + theUnsafe.putLong(((DirectBuffer) buf).address() + offset, val); + } else { + assert buf.hasArray(); + theUnsafe.putLong(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, val); + } + return offset + Bytes.SIZEOF_LONG; + } + + static int getInt(ByteBuffer buf, int offset) { + if (buf.isDirect()) { + return theUnsafe.getInt(((DirectBuffer) buf).address() + offset); + } else { + assert buf.hasArray(); + return theUnsafe.getInt(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset); + } + } + + static byte get(ByteBuffer buf, int offset) { + if (buf.isDirect()) { + return theUnsafe.getByte(((DirectBuffer) buf).address() + offset); + } else { + assert buf.hasArray(); + return theUnsafe.getByte(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset); + } + } + + static int putByte(ByteBuffer buf, int offset, byte b) { + if (buf.isDirect()) { + UnsafeUtil.theUnsafe.putByte(((DirectBuffer) buf).address() + offset, b); + } else { + assert buf.hasArray(); + UnsafeUtil.theUnsafe.putByte(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + + offset, b); + } + return offset + 1; + } + + static void copy(ByteBuffer src, int srcOffset, ByteBuffer dest, int destOffset, int length) { + long srcAddress = srcOffset; + Object srcBase = null; + if (src.isDirect()) { + srcAddress = srcAddress + ((DirectBuffer) src).address(); + } else { + srcAddress = srcAddress + BYTE_ARRAY_BASE_OFFSET + src.arrayOffset(); + srcBase = src.array(); + } + long destAddress = destOffset; + Object destBase = null; + if (dest.isDirect()) { + destAddress = destAddress + ((DirectBuffer) dest).address(); + } else { + destAddress = destAddress + BYTE_ARRAY_BASE_OFFSET + dest.arrayOffset(); + destBase = dest.array(); + } + UnsafeUtil.theUnsafe.copyMemory(srcBase, srcAddress, destBase, destAddress, length); + } + + static void copy(ByteBuffer src, int srcOffset, byte[] dest, int destOffset, int length) { + long srcAddress = srcOffset; + Object srcBase = null; + if (src.isDirect()) { + srcAddress = srcAddress + ((DirectBuffer) src).address(); + } else { + srcAddress = srcAddress + BYTE_ARRAY_BASE_OFFSET + src.arrayOffset(); + srcBase = src.array(); + } + long destAddress = destOffset + BYTE_ARRAY_BASE_OFFSET; + UnsafeUtil.theUnsafe.copyMemory(srcBase, srcAddress, dest, destAddress, length); + } + + static void copy(byte[] src, int srcOffset, ByteBuffer dest, int destOffset, int length) { + long destAddress = destOffset; + Object destBase = null; + if (dest.isDirect()) { + destAddress = destAddress + ((DirectBuffer) dest).address(); + } else { + destAddress = destAddress + BYTE_ARRAY_BASE_OFFSET + dest.arrayOffset(); + destBase = dest.array(); + } + long srcAddress = srcOffset + BYTE_ARRAY_BASE_OFFSET; + UnsafeUtil.theUnsafe.copyMemory(src, srcAddress, destBase, destAddress, length); + } + + static short getShort(ByteBuffer buf, int offset) { + if (buf.isDirect()) { + return theUnsafe.getShort(((DirectBuffer) buf).address() + offset); + } else { + assert buf.hasArray(); + return theUnsafe.getShort(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset); + } + } + + static int toIntUnsafe(ByteBuffer buf, int offset) { + if (UnsafeUtil.littleEndian) { + return Integer.reverseBytes(getInt(buf, offset)); + } else { + return getInt(buf, offset); + } + } + + static short toShortUnsafe(ByteBuffer buf, int offset) { + if (UnsafeUtil.littleEndian) { + return Short.reverseBytes(getShort(buf, offset)); + } else { + return getShort(buf, offset); + } + } + + static long toLongUnsafe(ByteBuffer buf, int offset) { + if (UnsafeUtil.littleEndian) { + return Long.reverseBytes(getLong(buf, offset)); + } else { + return getLong(buf, offset); + } + } + + static int putShortUnsafe(ByteBuffer buffer, int offset, short val) { + if (UnsafeUtil.littleEndian) { + val = Short.reverseBytes(val); + } + return putShort(buffer, offset, val); + } + + static int putIntUnsafe(ByteBuffer buffer, int offset, int val) { + if (UnsafeUtil.littleEndian) { + val = Integer.reverseBytes(val); + } + return putInt(buffer, offset, val); + } + + static int putLongUnsafe(ByteBuffer buffer, int offset, long val) { + if (UnsafeUtil.littleEndian) { + val = Long.reverseBytes(val); + } + return putLong(buffer, offset, val); + } + + static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) { + final int minLength = Math.min(l1, l2); + final int minWords = minLength / Bytes.SIZEOF_LONG; + + /* + * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a + * time is no slower than comparing 4 bytes at a time even on 32-bit. On + * the other hand, it is substantially faster on 64-bit. + */ + // TODO *8 can be replaced by <<3. Same in Bytes class also + for (int i = 0; i < minWords * Bytes.SIZEOF_LONG; i += Bytes.SIZEOF_LONG) { + long lw = getLong(buf1, o1 + i); + long rw = getLong(buf2, o2 + i); + long diff = lw ^ rw; + if (diff != 0) { + return lessThanUnsignedLong(lw, rw) ? -1 : 1; + } + } + int offset = minWords * Bytes.SIZEOF_LONG; + + if (minLength - offset >= Bytes.SIZEOF_INT) { + int il = getInt(buf1, o1 + offset); + int ir = getInt(buf2, o2 + offset); + if (il != ir) { + return lessThanUnsignedInt(il, ir) ? -1 : 1; + } + offset += Bytes.SIZEOF_INT; + } + if (minLength - offset >= Bytes.SIZEOF_SHORT) { + short sl = getShort(buf1, o1 + offset); + short sr = getShort(buf2, o2 + offset); + if (sl != sr) { + return lessThanUnsignedShort(sl, sr) ? -1 : 1; + } + offset += Bytes.SIZEOF_SHORT; + } + if (minLength - offset == 1) { + int a = (get(buf1, o1 + offset) & 0xff); + int b = (get(buf2, o2 + offset) & 0xff); + if (a != b) { + return a - b; + } + } + return l1 - l2; + } + + // Fix should be : already reverse endian is done in the bytes.toIntUnsafe + // and in the lessThanUnsignedInt we are again doing reverseEndian and hence the problem + static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) { + final int minLength = Math.min(l1, l2); + final int minWords = minLength / Bytes.SIZEOF_LONG; + final long offset2Adj = o2 + BYTE_ARRAY_BASE_OFFSET; + for (int i = 0; i < minWords * Bytes.SIZEOF_LONG; i += Bytes.SIZEOF_LONG) { + long lw = getLong(buf1, o1 + i); + long rw = theUnsafe.getLong(buf2, (long)offset2Adj + i); + long diff = lw ^ rw; + if (diff != 0) { + return lessThanUnsignedLong(lw, rw) ? -1 : 1; + } + } + int offset = minWords * Bytes.SIZEOF_LONG; + + if (minLength - offset >= Bytes.SIZEOF_INT) { + int il = getInt(buf1, o1 + offset); + int ir = theUnsafe.getInt(buf2, offset2Adj + offset); + if (il != ir) { + return lessThanUnsignedInt(il, ir) ? -1 : 1; + } + offset += Bytes.SIZEOF_INT; + } + if (minLength - offset >= Bytes.SIZEOF_SHORT) { + short sl = getShort(buf1, o1 + offset); + short sr = theUnsafe.getShort(buf2, offset2Adj + offset); + if (sl != sr) { + return lessThanUnsignedShort(sl, sr) ? -1 : 1; + } + offset += Bytes.SIZEOF_SHORT; + } + if (minLength - offset == 1) { + int a = (buf1.get(o1 + offset) & 0xff); + int b = (buf2[o2 + offset] & 0xff); + if (a != b) { + return a - b; + } + } + return l1 - l2; + } + + // Anoop : Why this is added here when it is already present in ByteArrayUtils + static int compareTo(byte[] buf1, int o1, int l1, byte[] buf2, int o2, int l2) { + final int minLength = Math.min(l1, l2); + final int minWords = minLength / Bytes.SIZEOF_LONG; + final long offset1Adj = o1 + BYTE_ARRAY_BASE_OFFSET; + final long offset2Adj = o2 + BYTE_ARRAY_BASE_OFFSET; + for (int i = 0; i < minWords * Bytes.SIZEOF_LONG; i += Bytes.SIZEOF_LONG) { + long lw = theUnsafe.getLong(buf1, (long)offset1Adj + i); + long rw = theUnsafe.getLong(buf2, (long)offset2Adj + i); + long diff = lw ^ rw; + if (diff != 0) { + return lessThanUnsignedLong(lw, rw) ? -1 : 1; + } + } + int offset = minWords * Bytes.SIZEOF_LONG; + + if (minLength - offset >= Bytes.SIZEOF_INT) { + int il = theUnsafe.getInt(buf1, offset1Adj + offset); + int ir = theUnsafe.getInt(buf2, offset2Adj + offset); + if (il != ir) { + return lessThanUnsignedInt(il, ir) ? -1 : 1; + } + offset += Bytes.SIZEOF_INT; + } + if (minLength - offset >= Bytes.SIZEOF_SHORT) { + short sl = theUnsafe.getShort(buf1, offset1Adj + offset); + short sr = theUnsafe.getShort(buf2, offset2Adj + offset); + if (sl != sr) { + return lessThanUnsignedShort(sl, sr) ? -1 : 1; + } + offset += Bytes.SIZEOF_SHORT; + } + if (minLength - offset == 1) { + int a = (buf1[o1 + offset] & 0xff); + int b = (buf2[o2 + offset] & 0xff); + if (a != b) { + return a - b; + } + } + return l1 - l2; + } + } } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java index bec35ee..f035165 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java @@ -46,6 +46,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.WritableComparator; @@ -743,12 +744,16 @@ public class Bytes implements Comparable { */ public static byte[] toBytes(long val) { byte [] b = new byte[8]; + toBytes(val, b); + return b; + } + + public static void toBytes(long val, byte[] out) { for (int i = 7; i > 0; i--) { - b[i] = (byte) val; + out[i] = (byte) val; val >>>= 8; } - b[0] = (byte) val; - return b; + out[0] = (byte) val; } /** @@ -942,13 +947,17 @@ public class Bytes implements Comparable { * @return the byte array */ public static byte[] toBytes(int val) { - byte [] b = new byte[4]; + byte[] b = new byte[4]; + toBytes(val, b); + return b; + } + + public static void toBytes(int val, byte[] out) { for(int i = 3; i > 0; i--) { - b[i] = (byte) val; + out[i] = (byte) val; val >>>= 8; } - b[0] = (byte) val; - return b; + out[0] = (byte) val; } /** @@ -1115,12 +1124,16 @@ public class Bytes implements Comparable { */ public static byte[] toBytes(short val) { byte[] b = new byte[SIZEOF_SHORT]; - b[1] = (byte) val; - val >>= 8; - b[0] = (byte) val; + toBytes(val, b); return b; } + public static void toBytes(short val, byte[] out) { + out[1] = (byte) val; + val >>= 8; + out[0] = (byte) val; + } + /** * Converts a byte array to a short value * @param bytes byte array @@ -2066,6 +2079,51 @@ public class Bytes implements Comparable { /** * Binary search for keys in indexes. + * + * @param arr + * array of byte arrays to search for + * @param key + * the key you want to find + * @param offset + * the offset in the key you want to find + * @param length + * the length of the key + * @param comparator + * a comparator to compare. + * @return zero-based index of the key, if the key is present in the array. + * Otherwise, a value -(i + 1) such that the key is between arr[i - 1] + * and arr[i] non-inclusively, where i is in [0, i], if we define + * arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above + * means that this function can return 2N + 1 different values ranging + * from -(N + 1) to N - 1. + */ + public static int binarySearch(byte[][] arr, ByteBuffer key, int offset, int length, + RawComparator comparator) { + int low = 0; + int high = arr.length - 1; + + while (low <= high) { + int mid = (low + high) >>> 1; + // we have to compare in this order, because the comparator order + // has special logic when the 'left side' is a special key. + // FIXME : Just verify once. Using BytebufferUtils compare here + assert key.hasArray(); + int cmp = ByteBufferUtils.compareTo(key, offset, length, arr[mid], 0, arr[mid].length); + // key lives above the midpoint + if (cmp > 0) + low = mid + 1; + // key lives below the midpoint + else if (cmp < 0) + high = mid - 1; + // BAM. how often does this really happen? + else + return mid; + } + return -(low + 1); + } + + /** + * Binary search for keys in indexes. * * @param arr array of byte arrays to search for * @param key the key you want to find diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java index 82cf5c4..0f179ea 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -106,6 +108,10 @@ public abstract class Hash { public int hash(byte[] bytes) { return hash(bytes, bytes.length, -1); } + + public int hash(ByteBuffer bytes) { + return hash(bytes, bytes.capacity(), -1); + } /** * Calculate a hash using all bytes from the input argument, @@ -117,6 +123,17 @@ public abstract class Hash { public int hash(byte[] bytes, int initval) { return hash(bytes, 0, bytes.length, initval); } + + /** + * Calculate a hash using all bytes from the input argument, + * and a provided seed value. + * @param bytes input bytes + * @param initval seed value + * @return hash value + */ + public int hash(ByteBuffer bytes, int initval) { + return hash(bytes, 0, bytes.capacity(), initval); + } /** * Calculate a hash using bytes from 0 to length, and @@ -129,6 +146,18 @@ public abstract class Hash { public int hash(byte[] bytes, int length, int initval) { return hash(bytes, 0, length, initval); } + + /** + * Calculate a hash using bytes from 0 to length, and + * the provided seed value + * @param bytes input bytes + * @param length length of the valid bytes after offset to consider + * @param initval seed value + * @return hash value + */ + public int hash(ByteBuffer bytes, int length, int initval) { + return hash(bytes, 0, length, initval); + } /** * Calculate a hash using bytes from offset to offset + @@ -140,4 +169,15 @@ public abstract class Hash { * @return hash value */ public abstract int hash(byte[] bytes, int offset, int length, int initval); + + /** + * Calculate a hash using bytes from offset to offset + + * length, and the provided seed value. + * @param bytes input bytes + * @param offset the offset into the array to start consideration + * @param length length of the valid bytes after offset to consider + * @param initval seed value + * @return hash value + */ + public abstract int hash(ByteBuffer bytes, int offset, int length, int initval); } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java index 359e7a9..f549258 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java @@ -23,6 +23,7 @@ import static java.lang.Integer.rotateLeft; import java.io.FileInputStream; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -238,6 +239,75 @@ public class JenkinsHash extends Hash { c ^= b; c -= rotateLeft(b, 24); return c; } + + @Override + @SuppressWarnings("fallthrough") + public int hash(ByteBuffer key, int off, int nbytes, int initval) { + int length = nbytes; + int a, b, c; + a = b = c = 0xdeadbeef + length + initval; + int offset = off; + for (; length > 12; offset += 12, length -= 12) { + a += (ByteBufferUtils.getByte(key,offset) & BYTE_MASK); + a += ((ByteBufferUtils.getByte(key,offset + 1) & BYTE_MASK) << 8); + a += ((ByteBufferUtils.getByte(key,offset + 2) & BYTE_MASK) << 16); + a += ((ByteBufferUtils.getByte(key,offset + 3) & BYTE_MASK) << 24); + b += (ByteBufferUtils.getByte(key,offset + 4) & BYTE_MASK); + b += ((ByteBufferUtils.getByte(key,offset + 5) & BYTE_MASK) << 8); + b += ((ByteBufferUtils.getByte(key,offset + 6) & BYTE_MASK) << 16); + b += ((ByteBufferUtils.getByte(key,offset + 7) & BYTE_MASK) << 24); + c += (ByteBufferUtils.getByte(key,offset + 8) & BYTE_MASK); + c += ((ByteBufferUtils.getByte(key,offset + 9) & BYTE_MASK) << 8); + c += ((ByteBufferUtils.getByte(key,offset + 10) & BYTE_MASK) << 16); + c += ((ByteBufferUtils.getByte(key,offset + 11) & BYTE_MASK) << 24); + a -= c; a ^= rotateLeft(c, 4); c += b; + b -= a; b ^= rotateLeft(a, 6); a += c; + c -= b; c ^= rotateLeft(b, 8); b += a; + a -= c; a ^= rotateLeft(c, 16); c += b; + b -= a; b ^= rotateLeft(a, 19); a += c; + c -= b; c ^= rotateLeft(b, 4); b += a; + } + + //-------------------------------- last block: affect all 32 bits of (c) + switch (length) { // all the case statements fall through + case 12: + c += ((ByteBufferUtils.getByte(key,offset + 11) & BYTE_MASK) << 24); + case 11: + c += ((ByteBufferUtils.getByte(key,offset + 10) & BYTE_MASK) << 16); + case 10: + c += ((ByteBufferUtils.getByte(key,offset) & BYTE_MASK) << 8); + case 9: + c += (ByteBufferUtils.getByte(key,offset + 8) & BYTE_MASK); + case 8: + b += ((ByteBufferUtils.getByte(key,offset + 7) & BYTE_MASK) << 24); + case 7: + b += ((ByteBufferUtils.getByte(key,offset + 6) & BYTE_MASK) << 16); + case 6: + b += ((ByteBufferUtils.getByte(key,offset + 5) & BYTE_MASK) << 8); + case 5: + b += (ByteBufferUtils.getByte(key,offset + 4) & BYTE_MASK); + case 4: + a += ((ByteBufferUtils.getByte(key,offset + 3) & BYTE_MASK) << 24); + case 3: + a += ((ByteBufferUtils.getByte(key,offset + 2) & BYTE_MASK) << 16); + case 2: + a += ((ByteBufferUtils.getByte(key,offset + 1) & BYTE_MASK) << 8); + case 1: + //noinspection PointlessArithmeticExpression + a += (ByteBufferUtils.getByte(key,offset + 0) & BYTE_MASK); + break; + case 0: + return c; + } + c ^= b; c -= rotateLeft(b, 14); + a ^= c; a -= rotateLeft(c, 11); + b ^= a; b -= rotateLeft(a, 25); + c ^= b; c -= rotateLeft(b, 16); + a ^= c; a -= rotateLeft(c, 4); + b ^= a; b -= rotateLeft(a, 14); + c ^= b; c -= rotateLeft(b, 24); + return c; + } /** * Compute the hash of the specified file diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/MultiByteBuffer.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/MultiByteBuffer.java new file mode 100644 index 0000000..0b1296c --- /dev/null +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/MultiByteBuffer.java @@ -0,0 +1,858 @@ +/** + * 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.util; + +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.WritableUtils; + +public class MultiByteBuffer { + + static final Log LOG = LogFactory.getLog(MultiByteBuffer.class); + private final ByteBuffer[] items; + private ByteBuffer curItem = null; + private int curItemIndex = 0; + private final boolean singleItem; + private int limit = 0; + private int markedIndex = -1; + // Read only structures + private final TreeMap offsetMap; + private final int[] itemBeginPos; + private Integer limitedIndex; + + public MultiByteBuffer(ByteBuffer... items) { + assert items != null; + assert items.length > 0; + this.items = items; + this.curItem = this.items[this.curItemIndex]; + this.singleItem = items.length == 1; + offsetMap = new TreeMap(); + // See below optimization in getInt(int) where we check whether the given index land in current + // item. For this we need to check whether the passed index is less than the next item begin + // offset. To handle this effectively for the last item buffer, we add an extra item into this + // array. + itemBeginPos = new int[items.length + 1]; + int offset = 0; + for (int i = 0; i < items.length; i++) { + ByteBuffer item = items[i]; + // items[i] = item.slice(); + item.rewind(); + itemBeginPos[i] = offset; + offsetMap.put(offset, i); + int l = item.limit() - item.position(); + offset += l; + } + this.limit = offset; + this.itemBeginPos[items.length] = offset + 1; + this.limitedIndex = this.items.length - 1; + } + + private MultiByteBuffer(ByteBuffer[] items, TreeMap offsetMap, + int[] itemBeginPos, int limit, int limitedIndex, int curItemIndex, int markedIndex) { + this.items = items; + this.curItemIndex = curItemIndex; + this.curItem = this.items[this.curItemIndex]; + this.singleItem = items.length == 1; + this.offsetMap = offsetMap; + this.itemBeginPos = itemBeginPos; + this.limit = limit; + this.limitedIndex = limitedIndex; + this.markedIndex = markedIndex; + } + + public byte[] array() { + if (hasArray()) { + return this.curItem.array(); + } + throw new UnsupportedOperationException(); + } + + public int arrayOffset() { + if (hasArray()) { + return this.curItem.arrayOffset(); + } + throw new UnsupportedOperationException(); + } + + public boolean hasArray() { + return this.singleItem && this.curItem.hasArray(); + } + + public int capacity() { + int c = 0; + for (ByteBuffer item : this.items) { + c += item.capacity(); + } + return c; + } + + public byte get(int index) { + if (singleItem) { + return ByteBufferUtils.getByte(this.curItem, index); + } + Entry floorEntry = this.offsetMap.floorEntry(index); + if (floorEntry != null) { + ByteBuffer item = items[floorEntry.getValue()]; + return ByteBufferUtils.getByte(item, index - floorEntry.getKey()); + } + throw new IndexOutOfBoundsException("Specified index " + index + " is outside the boundary"); + } + + public short getShort(int index) { + if (singleItem) { + return ByteBufferUtils.getShort(curItem, index); + } + // Mostly the index specified will land within this current item. Short circuit for that + if (this.itemBeginPos[this.curItemIndex] <= index + && this.itemBeginPos[this.curItemIndex + 1] > index) { + int relPos = index - this.itemBeginPos[this.curItemIndex]; + if (this.curItem.limit() - relPos >= Bytes.SIZEOF_SHORT) { + return ByteBufferUtils.getShort(this.curItem, relPos); + } + } + Entry floorEntry = this.offsetMap.floorEntry(index); + if (floorEntry != null) { + int beginOffset = floorEntry.getKey(); + ByteBuffer item = items[floorEntry.getValue()]; + int offsetInItem = index - beginOffset; + int remainingLen = item.capacity() - offsetInItem; + if (remainingLen >= Bytes.SIZEOF_SHORT) { + return ByteBufferUtils.getShort(item, offsetInItem); + } + if (items.length - 1 == floorEntry.getValue()) { + // means cur item is the last one and we wont be able to read a int. Throw exception + throw new BufferUnderflowException(); + } + ByteBuffer nextItem = items[floorEntry.getValue() + 1]; + // Get available one byte from this item and remaining one from next + short n = 0; + n ^= ByteBufferUtils.getByte(item, offsetInItem) & 0xFF; + n <<= 8; + n ^= ByteBufferUtils.getByte(nextItem, 0) & 0xFF; + return n; + } + throw new IndexOutOfBoundsException("Specified index " + index + " is outside the boundary"); + } + + public int getInt(int index) { + if (singleItem) { + return ByteBufferUtils.getInt(this.curItem, index); + } + // Mostly the index specified will land within this current item. Short circuit for that + if (this.itemBeginPos[this.curItemIndex] <= index + && this.itemBeginPos[this.curItemIndex + 1] > index) { + int relPos = index - this.itemBeginPos[this.curItemIndex]; + if (this.curItem.limit() - relPos >= Bytes.SIZEOF_INT) { + return ByteBufferUtils.getInt(this.curItem, relPos); + } + } + return getIntUsingMap(index); + } + + // TODO add javadoc.. diff btw getInt(int) + public int getIntStrictlyForward(int index) { + if (singleItem) { + return ByteBufferUtils.getInt(this.curItem, index); + } + // Mostly the index specified will land within this current item. Short circuit for that + if (this.itemBeginPos[this.curItemIndex + 1] > index) { + int relPos = index - this.itemBeginPos[this.curItemIndex]; + if (this.curItem.limit() - relPos >= Bytes.SIZEOF_INT) { + return ByteBufferUtils.getInt(this.curItem, relPos); + } + } + return getIntUsingMap(index); + } + + private int getIntUsingMap(int index) { + Entry floorEntry = this.offsetMap.floorEntry(index); + if (floorEntry != null) { + int beginOffset = floorEntry.getKey(); + ByteBuffer item = items[floorEntry.getValue()]; + int offsetInItem = index - beginOffset; + int remainingLen = item.capacity() - offsetInItem; + if (remainingLen >= Bytes.SIZEOF_INT) { + return ByteBufferUtils.getInt(item, offsetInItem); + } + if (items.length - 1 == floorEntry.getValue()) { + // means cur item is the last one and we wont be able to read a int. Throw exception + throw new BufferUnderflowException(); + } + ByteBuffer nextItem = items[floorEntry.getValue() + 1]; + // Get available bytes from this item and remaining from next + int l = 0; + for (int i = offsetInItem; i < item.capacity(); i++) { + l <<= 8; + l ^= ByteBufferUtils.getByte(item, i) & 0xFF; + } + for (int i = 0; i < Bytes.SIZEOF_INT - remainingLen; i++) { + l <<= 8; + l ^= ByteBufferUtils.getByte(nextItem, i) & 0xFF; + } + return l; + } + throw new IndexOutOfBoundsException("Specified index " + index + " is outside the boundary"); + } + + public long getLong(int index) { + if (singleItem) { + return this.curItem.getLong(index); + } + if (this.itemBeginPos[this.curItemIndex] <= index + && this.itemBeginPos[this.curItemIndex + 1] > index) { + int relPos = index - this.itemBeginPos[this.curItemIndex]; + if (this.curItem.limit() - relPos >= Bytes.SIZEOF_LONG) { + return ByteBufferUtils.getLong(this.curItem, relPos); + } + } + Entry floorEntry = this.offsetMap.floorEntry(index); + if (floorEntry != null) { + int beginOffset = floorEntry.getKey(); + ByteBuffer item = items[floorEntry.getValue()]; + int offsetInItem = index - beginOffset; + int remainingLen = item.capacity() - offsetInItem; + if (remainingLen >= Bytes.SIZEOF_LONG) { + return ByteBufferUtils.getLong(item, offsetInItem); + } + if (items.length - 1 == floorEntry.getValue()) { + // means cur item is the last one and we wont be able to read a long. Throw exception + throw new BufferUnderflowException(); + } + ByteBuffer nextItem = items[floorEntry.getValue() + 1]; + // Get available bytes from this item and remaining from next + long l = 0; + for (int i = offsetInItem; i < item.capacity(); i++) { + l <<= 8; + l ^= ByteBufferUtils.getByte(item, i) & 0xFF; + } + for (int i = 0; i < Bytes.SIZEOF_LONG - remainingLen; i++) { + l <<= 8; + l ^= ByteBufferUtils.getByte(nextItem, i) & 0xFF; + } + return l; + } + throw new IndexOutOfBoundsException("Specified index " + index + " is outside the boundary"); + } + + public int position() { + if (this.singleItem) return this.curItem.position(); + return itemBeginPos[this.curItemIndex] + this.curItem.position(); + } + + public MultiByteBuffer position(int position) { + if (this.singleItem) { + this.curItem.position(position); + return this; + } + // Short circuit for positioning within the cur item. Mostly that is the case. + if (this.itemBeginPos[this.curItemIndex] <= position + && this.itemBeginPos[this.curItemIndex + 1] > position) { + this.curItem.position(position - this.itemBeginPos[this.curItemIndex]); + return this; + } + return positionUsingMap(position); + } + + private MultiByteBuffer positionUsingMap(int position) { + // TODO for all other in btw items, set position to begin + Entry floorEntry = this.offsetMap.floorEntry(position); + if (floorEntry != null) { + this.curItemIndex = floorEntry.getValue(); + this.curItem = this.items[this.curItemIndex]; + this.curItem.position(position - floorEntry.getKey()); + return this; + } + throw new IndexOutOfBoundsException("Specified position " + position + + " is outside the boundary"); + } + + public MultiByteBuffer moveBack(int length) { + while (length != 0) { + if (length > curItem.position()) { + length -= curItem.position(); + this.curItem.position(0); + this.curItemIndex--; + this.curItem = this.items[curItemIndex]; + } else { + this.curItem.position(curItem.position() - length); + return this; + } + } + return this; + } + + public MultiByteBuffer rewind() { + // All items after the marked position upto the current item should be reset to 0 + for (int i = 0; i < this.items.length; i++) { + this.items[i].rewind(); + } + this.curItemIndex = 0; + this.curItem = this.items[this.curItemIndex]; + markedIndex = -1; + return this; + } + + public MultiByteBuffer mark() { + this.markedIndex = this.curItemIndex; + this.curItem.mark(); + return this; + } + + public MultiByteBuffer reset() { + // when the buffer is moved to the next one.. the reset should happen on the previous marked item + // and the new one should be taken as the base + ByteBuffer markedItem = this.items[this.markedIndex]; + markedItem.reset(); + this.curItem = markedItem; + // All items after the marked position upto the current item should be reset to 0 + for(int i = this.curItemIndex; i > this.markedIndex; i--) { + this.items[i].position(0); + } + this.curItemIndex = this.markedIndex; + return this; + } + + public int remaining() { + int remain = 0; + for (int i = curItemIndex; i < items.length; i++) { + remain += items[i].remaining(); + } + return remain; + } + + public final boolean hasRemaining() { + return this.curItem.hasRemaining() || this.curItemIndex < this.items.length - 1; + } + + public byte get() { + if (!singleItem && this.curItem.remaining() == 0) { + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + } + return this.curItem.get(); + } + + public short getShort() { + int remaining = -1; + if (singleItem || (remaining = this.curItem.remaining()) >= Bytes.SIZEOF_SHORT) { + return this.curItem.getShort(); + } + if (remaining == 0) { + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + return this.curItem.getShort(); + } + short n = 0; + n ^= get() & 0xFF; + n <<= 8; + n ^= get() & 0xFF; + return n; + } + + public int getInt() { + int remaining = -1; + if (singleItem || (remaining = this.curItem.remaining()) >= Bytes.SIZEOF_INT) { + return this.curItem.getInt(); + } + if (remaining == 0) { + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + return this.curItem.getInt(); + } + // Get available bytes from this item and remaining from next + int n = 0; + for (int i = 0; i < Bytes.SIZEOF_INT; i++) { + n <<= 8; + n ^= get() & 0xFF; + } + return n; + } + + public void skip(int length) { + // Get available bytes from this item and remaining from next + int jump = 0; + while (true) { + jump = this.curItem.remaining(); + if (jump >= length) { + this.curItem.position(this.curItem.position() + length); + break; + } + this.curItem.position(this.curItem.position() + jump); + length -= jump; + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + } + } + + public long getLong() { + int remaining = -1; + if (singleItem || (remaining = this.curItem.remaining()) >= Bytes.SIZEOF_LONG) { + return this.curItem.getLong(); + } + if (remaining == 0) { + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + return this.curItem.getLong(); + } + // Get available bytes from this item and remaining from next + long l = 0; + for (int i = 0; i < Bytes.SIZEOF_LONG; i++) { + l <<= 8; + l ^= get() & 0xFF; + } + return l; + } + + public long getVLong() { + byte firstByte = get(); + int len = WritableUtils.decodeVIntSize(firstByte); + if (len == 1) { + return firstByte; + } + long i = 0; + for (int idx = 0; idx < len-1; idx++) { + byte b = get(); + i = i << 8; + i = i | (b & 0xFF); + } + return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i); + } + public MultiByteBuffer get(byte[] dst) { + return get(dst, 0, dst.length); + } + + public MultiByteBuffer get(byte[] dst, int offset, int length) { + if (this.singleItem) { + // Convert into BB Utils and use absolute gets + this.curItem.get(dst, offset, length); + } else { + while (length > 0) { + int toRead = Math.min(length, this.curItem.remaining()); + //Convert into BB Utils and use absolute gets + this.curItem.get(dst, offset, toRead); + length -= toRead; + if (length == 0) break; + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + offset += toRead; + } + } + return this; + } + + public MultiByteBuffer limit(int limit) { + this.limit = limit; + if (singleItem) { + this.curItem.limit(limit); + return this; + } + // Normally the limit will try to limit within the last BB item + int limitedIndexBegin = this.itemBeginPos[this.limitedIndex]; + if (limit >= limitedIndexBegin && limit < this.itemBeginPos[this.limitedIndex + 1]) { + this.items[this.limitedIndex].limit(limit - limitedIndexBegin); + return this; + } + Entry floorEntry = this.offsetMap.floorEntry(limit); + if (floorEntry != null) { + int beginOffset = floorEntry.getKey(); + int offsetInItem = limit - beginOffset; + ByteBuffer item = items[floorEntry.getValue()]; + item.limit(offsetInItem); + for (int i = this.limitedIndex; i < floorEntry.getValue(); i++) { + this.items[i].limit(this.items[i].capacity()); + } + this.limitedIndex = floorEntry.getValue(); + for (int i = floorEntry.getValue() + 1; i < this.items.length; i++) { + this.items[i].limit(this.items[i].position()); + } + return this; + } + throw new IndexOutOfBoundsException("Specified limit " + limit + " is outside the boundary"); + } + + public int limit() { + return this.limit; + } + + public MultiByteBuffer slice() { + if (this.singleItem) { + return new MultiByteBuffer(curItem.slice()); + } + ByteBuffer[] copy = new ByteBuffer[this.limitedIndex-this.curItemIndex+1]; + for (int i = curItemIndex,j=0; i <= this.limitedIndex; i++,j++) { + copy[j] = this.items[i].slice(); + } + return new MultiByteBuffer(copy); + } + + public MultiByteBuffer duplicate() { + if (this.singleItem) { + return new MultiByteBuffer(new ByteBuffer[] { curItem.duplicate() }, this.offsetMap, + this.itemBeginPos, this.limit, this.limitedIndex, this.curItemIndex, this.markedIndex); + } + ByteBuffer[] itemsCopy = new ByteBuffer[this.items.length]; + for (int i = 0; i < this.items.length; i++) { + itemsCopy[i] = items[i].duplicate(); + } + return new MultiByteBuffer(itemsCopy, this.offsetMap, this.itemBeginPos, this.limit, + this.limitedIndex, this.curItemIndex, this.markedIndex); + } + + public MultiByteBuffer put(byte b) { + if (!singleItem && this.curItem.remaining() == 0) { + this.curItemIndex++; + this.curItem = this.items[this.curItemIndex]; + } + this.curItem.put(b); + return this; + } + + public MultiByteBuffer put(int index, byte b) { + if (this.singleItem) { + this.curItem.put(index, b); + return this; + } + Entry floorEntry = this.offsetMap.floorEntry(index); + if (floorEntry != null) { + ByteBuffer item = items[floorEntry.getValue()]; + item.put(index - floorEntry.getKey(), b); + return this; + } + throw new IndexOutOfBoundsException("Specified index " + index + " is outside the boundary"); + } + + public void put(int offset, MultiByteBuffer src, int srcOffset, int length) { + if (src.hasArray() && this.hasArray()) { + System.arraycopy(src.array(), srcOffset + src.arrayOffset(), this.array(), this.arrayOffset() + + offset, length); + } else { + Entry feDest = this.offsetMap.floorEntry(offset); + Entry feSrc = src.offsetMap.floorEntry(srcOffset); + if (feDest != null && feSrc != null) { + int destIndex = feDest.getValue(); + ByteBuffer destItem = this.items[destIndex]; + offset = offset - feDest.getKey(); + + int srcIndex = feSrc.getValue(); + ByteBuffer srcItem = src.items[srcIndex]; + srcOffset = srcOffset - feSrc.getKey(); + int toRead, toWrite, toMove; + while (length > 0) { + toWrite = destItem.limit() - offset; + toRead = srcItem.limit() - srcOffset; + toMove = Math.min(length, Math.min(toRead, toWrite)); + ByteBufferUtils.copyFromBufferToBuffer(destItem, srcItem, srcOffset, offset, toMove); + length -= toMove; + if (length == 0) + break; + if (toRead < toWrite) { + srcItem = src.items[++srcIndex]; + srcOffset = 0; + offset += toMove; + } else if (toRead > toWrite) { + destItem = this.items[++destIndex]; + offset = 0; + srcOffset += toMove; + } else { + // toRead = toWrite case + srcItem = src.items[++srcIndex]; + srcOffset = 0; + destItem = this.items[++destIndex]; + offset = 0; + } + } + } else { + throw new IndexOutOfBoundsException(); + } + } + } + + public MultiByteBuffer putInt(int val) { + if (singleItem || this.curItem.remaining() >= Bytes.SIZEOF_INT) { + this.curItem.putInt(val); + return this; + } + // TODO handle both endian? + put(int3(val)); + put(int2(val)); + put(int1(val)); + put(int0(val)); + return this; + } + + private static byte int3(int x) { + return (byte) (x >> 24); + } + + private static byte int2(int x) { + return (byte) (x >> 16); + } + + private static byte int1(int x) { + return (byte) (x >> 8); + } + + private static byte int0(int x) { + return (byte) (x); + } + + public final MultiByteBuffer put(byte[] src) { + return put(src, 0, src.length); + } + + public MultiByteBuffer put(byte[] src, int offset, int length) { + if (singleItem || this.curItem.remaining() >= length) { + ByteBufferUtils.copyFromArrayToBuffer(this.curItem, src, offset, length); + return this; + } + int end = offset + length; + for (int i = offset; i < end; i++) { + this.put(src[i]); + } + return this; + } + + public MultiByteBuffer putLong(long val) { + if (singleItem || this.curItem.remaining() >= Bytes.SIZEOF_LONG) { + this.curItem.putLong(val); + return this; + } + // TODO handle both endian?? + put(long7(val)); + put(long6(val)); + put(long5(val)); + put(long4(val)); + put(long3(val)); + put(long2(val)); + put(long1(val)); + put(long0(val)); + return this; + } + + private static byte long7(long x) { + return (byte) (x >> 56); + } + + private static byte long6(long x) { + return (byte) (x >> 48); + } + + private static byte long5(long x) { + return (byte) (x >> 40); + } + + private static byte long4(long x) { + return (byte) (x >> 32); + } + + private static byte long3(long x) { + return (byte) (x >> 24); + } + + private static byte long2(long x) { + return (byte) (x >> 16); + } + + private static byte long1(long x) { + return (byte) (x >> 8); + } + + private static byte long0(long x) { + return (byte) (x); + } + + // TODO need neat javadoc + // TODO See who all use this.. Write down explicitly there that do not do relative gets as it + // returns underlying buffer. + public ByteBuffer asSubBuffer(int length) { + // as a single BB from cur position to length + // If this is within one BB item just return that BB after sliced properly + // If the length crosses multi items we will need create new Buffer as size + // 'length' and copy in the bytes + // a try to see if only duplicate and slice are really costly + if (this.singleItem || this.curItem.remaining() >= length) { + return this.curItem; + } + int offset = 0; + byte[] dupB = new byte[length]; + int locCurItemIndex = curItemIndex; + ByteBuffer locCurItem = curItem; + while (length > 0) { + int toRead = Math.min(length, locCurItem.remaining()); + // Convert into BB Utils and use absolute gets + ByteBufferUtils.copyFromBufferToByteArray(dupB, locCurItem, locCurItem.position(), offset, + toRead); + length -= toRead; + if (length == 0) break; + locCurItemIndex++; + locCurItem = this.items[locCurItemIndex]; + offset += toRead; + } + return ByteBuffer.wrap(dupB); + } + + /** + * @param offset + * @param length + * @return + */ + public Pair asSubBuffer(int offset, int length) { + if (this.singleItem) { + return new Pair(this.curItem, offset); + } + if (this.itemBeginPos[this.curItemIndex] <= offset) { + int relOffsetInCurItem = offset - this.itemBeginPos[this.curItemIndex]; + if (this.curItem.limit() - relOffsetInCurItem >= length) { + return new Pair(this.curItem, relOffsetInCurItem); + } + } + Entry floorEntry = this.offsetMap.floorEntry(offset); + if (floorEntry != null) { + int locCurItemIndex = floorEntry.getValue(); + ByteBuffer locCurItem = this.items[locCurItemIndex]; + offset = offset - floorEntry.getKey(); + if (locCurItem.limit() - offset >= length) { + return new Pair(locCurItem, offset); + } + byte[] dst = new byte[length]; + int destOffset = 0; + while (length > 0) { + int toRead = Math.min(length, locCurItem.limit() - offset); + ByteBufferUtils.copyFromBufferToByteArray(dst, locCurItem, offset, destOffset, toRead); + length -= toRead; + if (length == 0) + break; + locCurItemIndex++; + locCurItem = this.items[locCurItemIndex]; + destOffset += toRead; + offset = 0; + } + return new Pair(ByteBuffer.wrap(dst), 0); + } + throw new IndexOutOfBoundsException("Specified offset " + offset + " is outside the boundary"); + } + + public static String toStringBinary(MultiByteBuffer buf) { + if (buf == null) + return "null"; + if (buf.hasArray()) { + return Bytes.toStringBinary(buf.array(), buf.arrayOffset(), buf.capacity()); + } + return Bytes.toStringBinary(toBytes(buf)); + } + + private static byte[] toBytes(MultiByteBuffer buf) { + MultiByteBuffer dup = buf.duplicate(); + dup.position(0); + byte[] result = new byte[buf.remaining()]; + dup.get(result); + return result; + } + + public byte[] toBytes(int offset, int length) { + byte[] output = new byte[length]; + if (singleItem) { + ByteBufferUtils.copyFromBufferToByteArray(output, this.curItem, offset, 0, length); + return output; + } + Entry floorEntry = this.offsetMap.floorEntry(offset); + if (floorEntry != null) { + int index = floorEntry.getValue(); + ByteBuffer item = items[index]; + offset = offset - floorEntry.getKey(); + int toRead = item.limit() - offset; + int destinationOffset = 0; + while (length > 0) { + toRead = Math.min(length, toRead); + ByteBufferUtils.copyFromBufferToByteArray(output, item, offset, destinationOffset, toRead); + length -= toRead; + if (length == 0) break; + destinationOffset += toRead; + offset = 0; + item = items[++index]; + toRead = item.remaining(); + } + return output; + } + throw new IndexOutOfBoundsException("Specified offset " + offset + " is outside the boundary"); + } + + public static int compareTo(MultiByteBuffer buf1, int o1, int len1, MultiByteBuffer buf2, int o2, + int len2) { + if (buf1.hasArray() && buf2.hasArray()) { + return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(), + buf2.arrayOffset() + o2, len2); + } + int end1 = o1 + len1; + int end2 = o2 + len2; + for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) { + int a = buf1.get(i) & 0xFF; + int b = buf2.get(j) & 0xFF; + if (a != b) { + return a - b; + } + } + return len1 - len2; + } + + public static void copyFromBufferToBuffer(ByteBuffer out, MultiByteBuffer in, int sourceOffset, + int length) { + if (in.hasArray() && out.hasArray()) { + System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.position() + + out.arrayOffset(), length); + ByteBufferUtils.skip(out, length); + } else { + // Not used from real read path actually. So not going with optimization + for (int i = 0; i < length; ++i) { + out.put(in.get(sourceOffset + i)); + } + } + } + + public static int readCompressedInt(MultiByteBuffer buffer) { + byte b = buffer.get(); + if ((b & ByteBufferUtils.NEXT_BIT_MASK) != 0) { + return (b & ByteBufferUtils.VALUE_MASK) + + (readCompressedInt(buffer) << ByteBufferUtils.NEXT_BIT_SHIFT); + } + return b & ByteBufferUtils.VALUE_MASK; + } + + public static long readLong(MultiByteBuffer in, final int fitInBytes) { + long tmpLength = 0; + for (int i = 0; i < fitInBytes; ++i) { + tmpLength |= (in.get() & 0xffl) << (8l * i); + } + return tmpLength; + } + + @Override + public boolean equals(Object obj) { + if(!(obj instanceof MultiByteBuffer)) return false; + if(this == obj) return true; + MultiByteBuffer that = (MultiByteBuffer)obj; + if(this.capacity() != that.capacity()) return false; + if(compareTo(this, 0, this.capacity(), that, 0, this.capacity()) == 0) return true; + return false; + } +} diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java index 3663d41..45312af 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -89,4 +91,56 @@ public class MurmurHash extends Hash { return h; } + + @Override + public int hash(ByteBuffer data, int offset, int length, int seed) { + int m = 0x5bd1e995; + int r = 24; + + int h = seed ^ length; + + int len_4 = length >> 2; + + for (int i = 0; i < len_4; i++) { + int i_4 = (i << 2) + offset; + int k = ByteBufferUtils.getByte(data,i_4 + 3); + k = k << 8; + k = k | (ByteBufferUtils.getByte(data,i_4 + 2) & 0xff); + k = k << 8; + k = k | (ByteBufferUtils.getByte(data,i_4 + 1) & 0xff); + k = k << 8; + //noinspection PointlessArithmeticExpression + k = k | (ByteBufferUtils.getByte(data,i_4 + 0) & 0xff); + k *= m; + k ^= k >>> r; + k *= m; + h *= m; + h ^= k; + } + + // avoid calculating modulo + int len_m = len_4 << 2; + int left = length - len_m; + int i_m = len_m + offset; + + if (left != 0) { + if (left >= 3) { + h ^= ByteBufferUtils.getByte(data,i_m + 2) << 16; + } + if (left >= 2) { + h ^= ByteBufferUtils.getByte(data,i_m + 1) << 8; + } + if (left >= 1) { + h ^= ByteBufferUtils.getByte(data,i_m); + } + + h *= m; + } + + h ^= h >>> 13; + h *= m; + h ^= h >>> 15; + + return h; + } } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java index 89014db..8b59123 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.util; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import java.nio.ByteBuffer; /** * This is a very fast, non-cryptographic hash suitable for general hash-based @@ -90,4 +91,58 @@ public class MurmurHash3 extends Hash { return h1; } + + @Override + public int hash(ByteBuffer bytes, int offset, int length, int initval) { + final int c1 = 0xcc9e2d51; + final int c2 = 0x1b873593; + + int h1 = initval; + int roundedEnd = offset + (length & 0xfffffffc); // round down to 4 byte block + + for (int i = offset; i < roundedEnd; i += 4) { + // little endian load order + int k1 = (ByteBufferUtils.getByte(bytes, i) & 0xff) + | ((ByteBufferUtils.getByte(bytes, i + 1) & 0xff) << 8) + | ((ByteBufferUtils.getByte(bytes, i + 2) & 0xff) << 16) + | (ByteBufferUtils.getByte(bytes, i + 3) << 24); + k1 *= c1; + k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15); + k1 *= c2; + + h1 ^= k1; + h1 = (h1 << 13) | (h1 >>> 19); // ROTL32(h1,13); + h1 = h1 * 5 + 0xe6546b64; + } + + // tail + int k1 = 0; + + switch (length & 0x03) { + case 3: + k1 = (ByteBufferUtils.getByte(bytes,roundedEnd + 2) & 0xff) << 16; + // FindBugs SF_SWITCH_FALLTHROUGH + case 2: + k1 |= (ByteBufferUtils.getByte(bytes,roundedEnd + 1) & 0xff) << 8; + // FindBugs SF_SWITCH_FALLTHROUGH + case 1: + k1 |= (ByteBufferUtils.getByte(bytes,roundedEnd) & 0xff); + k1 *= c1; + k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15); + k1 *= c2; + h1 ^= k1; + } + + // finalization + h1 ^= length; + + // fmix(h1); + h1 ^= h1 >>> 16; + h1 *= 0x85ebca6b; + h1 ^= h1 >>> 13; + h1 *= 0xc2b2ae35; + h1 ^= h1 >>> 16; + + return h1; + } } diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java index 52bc4e0..fb132ca 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Random; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteBufferUtils; @@ -277,8 +278,8 @@ public class RedundantKVGenerator { } if (useTags) { - result.add(new KeyValue(row, family, qualifier, timestamp, value, new Tag[] { new Tag( - (byte) 1, "value1") })); + result.add(new KeyValue(row, family, qualifier, timestamp, value, + new Tag[] { new ArrayBackedTag((byte) 1, "value1") })); } else { result.add(new KeyValue(row, family, qualifier, timestamp, value)); } diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java index fea517f..b5fe120 100644 --- hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java +++ hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.*; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; @@ -29,6 +30,7 @@ import java.util.TreeMap; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; import org.junit.Test; @@ -89,6 +91,10 @@ public class TestCellUtil { } @Override + public boolean hasArray() { + return true; + } + @Override public byte[] getRowArray() { return this.row; } @@ -222,6 +228,36 @@ public class TestCellUtil { // TODO Auto-generated method stub return 0; } + + @Override + public ByteBuffer getRowBuffer() { + // TODO Auto-generated method stub + return null; + } + + @Override + public ByteBuffer getFamilyBuffer() { + // TODO Auto-generated method stub + return null; + } + + @Override + public ByteBuffer getQualifierBuffer() { + // TODO Auto-generated method stub + return null; + } + + @Override + public ByteBuffer getValueBuffer() { + // TODO Auto-generated method stub + return null; + } + + @Override + public ByteBuffer getTagsBuffer() { + // TODO Auto-generated method stub + return null; + } }; /** diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index 0b67b41..8efe9cf 100644 --- hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -517,7 +517,7 @@ public class TestKeyValue extends TestCase { byte[] metaValue1 = Bytes.toBytes("metaValue1"); byte[] metaValue2 = Bytes.toBytes("metaValue2"); KeyValue kv = new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, new Tag[] { - new Tag((byte) 1, metaValue1), new Tag((byte) 2, metaValue2) }); + new ArrayBackedTag((byte) 1, metaValue1), new ArrayBackedTag((byte) 2, metaValue2) }); assertTrue(kv.getTagsLength() > 0); assertTrue(Bytes.equals(kv.getRow(), row)); assertTrue(Bytes.equals(kv.getFamily(), cf)); @@ -529,44 +529,42 @@ public class TestKeyValue extends TestCase { boolean meta1Ok = false, meta2Ok = false; for (Tag tag : tags) { if (tag.getType() == (byte) 1) { - if (Bytes.equals(tag.getValue(), metaValue1)) { + if (Bytes.equals(TagUtil.getTagValue(tag), metaValue1)) { meta1Ok = true; } } else { - if (Bytes.equals(tag.getValue(), metaValue2)) { + if (Bytes.equals(TagUtil.getTagValue(tag), metaValue2)) { meta2Ok = true; } } } assertTrue(meta1Ok); assertTrue(meta2Ok); - Iterator tagItr = CellUtil.tagsIterator(kv.getTagsArray(), kv.getTagsOffset(), - kv.getTagsLength()); + Iterator tagItr = CellUtil.tagsIterator(kv); //Iterator tagItr = kv.tagsIterator(); assertTrue(tagItr.hasNext()); Tag next = tagItr.next(); assertEquals(10, next.getTagLength()); assertEquals((byte) 1, next.getType()); - Bytes.equals(next.getValue(), metaValue1); + Bytes.equals(TagUtil.getTagValue(next), metaValue1); assertTrue(tagItr.hasNext()); next = tagItr.next(); assertEquals(10, next.getTagLength()); assertEquals((byte) 2, next.getType()); - Bytes.equals(next.getValue(), metaValue2); + Bytes.equals(TagUtil.getTagValue(next), metaValue2); assertFalse(tagItr.hasNext()); - tagItr = CellUtil.tagsIterator(kv.getTagsArray(), kv.getTagsOffset(), - kv.getTagsLength()); + tagItr = CellUtil.tagsIterator(kv); assertTrue(tagItr.hasNext()); next = tagItr.next(); assertEquals(10, next.getTagLength()); assertEquals((byte) 1, next.getType()); - Bytes.equals(next.getValue(), metaValue1); + Bytes.equals(TagUtil.getTagValue(next), metaValue1); assertTrue(tagItr.hasNext()); next = tagItr.next(); assertEquals(10, next.getTagLength()); assertEquals((byte) 2, next.getType()); - Bytes.equals(next.getValue(), metaValue2); + Bytes.equals(TagUtil.getTagValue(next), metaValue2); assertFalse(tagItr.hasNext()); } diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffHeapBBBackedKeyValue.java hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffHeapBBBackedKeyValue.java new file mode 100644 index 0000000..1841bf4 --- /dev/null +++ hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffHeapBBBackedKeyValue.java @@ -0,0 +1,194 @@ +/** + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class }) +public class TestOffHeapBBBackedKeyValue { + private static final String QUAL2 = "qual2"; + private static final String FAM2 = "fam2"; + private static final String QUAL1 = "qual1"; + private static final String FAM1 = "fam1"; + private static final String ROW1 = "row1"; + private static ByteBuffer rowBuf; + private static ByteBuffer famBuf; + private static ByteBuffer qualBuf; + private static ByteBuffer fam2Buf; + private static ByteBuffer qual2Buf; + private static final ArrayBackedTag t1 = new ArrayBackedTag((byte) 1, Bytes.toBytes("TAG1")); + private static final ArrayBackedTag t2 = new ArrayBackedTag((byte) 2, Bytes.toBytes("TAG2")); + private static final ArrayList tags = new ArrayList(); + static { + tags.add(t1); + tags.add(t2); + } + + @BeforeClass + public static void setUp() { + init(); + } + @Test + public void testByteBufferBackedKeyValue() throws Exception { + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(rowBuf, famBuf, qualBuf, 0L, + Type.Put, rowBuf); + assertEquals(ROW1, + ByteBufferUtils.toStringBinary(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength())); + assertEquals( + FAM1, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + QUAL1, + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + assertEquals( + ROW1, + ByteBufferUtils.toStringBinary(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength())); + assertEquals(0L, kv.getTimestamp()); + assertEquals(Type.Put.getCode(), kv.getTypeByte()); + kv = new ByteBufferBackedKeyValue(rowBuf, fam2Buf, qual2Buf, 0L, Type.Put, rowBuf); + assertEquals( + FAM2, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + QUAL2, + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + byte[] nullQualifier = new byte[0]; + kv = new ByteBufferBackedKeyValue(rowBuf, famBuf, ByteBuffer.wrap(nullQualifier), 0L, Type.Put, + rowBuf); + assertEquals(ROW1, + ByteBufferUtils.toStringBinary(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength())); + assertEquals( + FAM1, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + "", + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + assertEquals( + ROW1, + ByteBufferUtils.toStringBinary(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength())); + assertEquals(0L, kv.getTimestamp()); + assertEquals(Type.Put.getCode(), kv.getTypeByte()); + } + + private static void init() { + byte[] row1 = Bytes.toBytes(ROW1); + rowBuf = ByteBuffer.allocateDirect(row1.length); + for(byte b : row1) { + rowBuf.put(b); + } + byte[] fam1 = Bytes.toBytes(FAM1); + famBuf = ByteBuffer.allocateDirect(fam1.length); + for(byte b : fam1) { + famBuf.put(b); + } + + byte[] fam2 = Bytes.toBytes(FAM2); + fam2Buf = ByteBuffer.allocateDirect(fam2.length); + for(byte b : fam2) { + fam2Buf.put(b); + } + + byte[] qual1 = Bytes.toBytes(QUAL1); + qualBuf = ByteBuffer.allocateDirect(qual1.length); + for(byte b : qual1) { + qualBuf.put(b); + } + + byte[] qual2 = Bytes.toBytes(QUAL2); + qual2Buf = ByteBuffer.allocateDirect(qual2.length); + for(byte b : qual2) { + qual2Buf.put(b); + } + } + + @Test + public void testByteBufferBackedKeyValueWithTags() throws Exception { + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(rowBuf, famBuf, qualBuf, 0L, + Type.Put, rowBuf, tags); + assertEquals(ROW1, + ByteBufferUtils.toStringBinary(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength())); + assertEquals( + FAM1, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + QUAL1, + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + assertEquals( + ROW1, + ByteBufferUtils.toStringBinary(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength())); + assertEquals(0L, kv.getTimestamp()); + assertEquals(Type.Put.getCode(), kv.getTypeByte()); + List resTags = TagUtil.asList(kv.getTagsBuffer(), kv.getTagsOffset(), kv.getTagsLength()); + Tag tag1 = resTags.get(0); + assertEquals(t1.getType(), tag1.getType()); + assertEquals(Bytes.toString(t1.getValue()), Bytes.toString(TagUtil.getTagValue(tag1))); + Tag tag2 = resTags.get(1); + assertEquals(tag2.getType(), tag2.getType()); + assertEquals(Bytes.toString(t2.getValue()), Bytes.toString(TagUtil.getTagValue(tag2))); + Tag res = TagUtil.getTag(kv.getTagsBuffer(), 0, kv.getTagsLength(), (byte) 2); + assertEquals(Bytes.toString(t2.getValue()), Bytes.toString(TagUtil.getTagValue(tag2))); + res = TagUtil.getTag(kv.getTagsBuffer(), 0, kv.getTagsLength(), (byte) 3); + assertNull(res); + } + + @Test + public void testGetKeyMethods() throws Exception { + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(rowBuf, famBuf, qualBuf, 0L, + Type.Put, rowBuf, tags); + ByteBuffer key = kv.getKey(); + assertEquals(key.capacity(), kv.getKeyLength()); + ByteBufferBackedKeyOnlyKeyValue keyOnlyKv = + new ByteBufferBackedKeyOnlyKeyValue(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength()); + assertEquals(ROW1, ByteBufferUtils.toStringBinary(keyOnlyKv.getRowBuffer(), + keyOnlyKv.getRowOffset(), keyOnlyKv.getRowLength())); + assertEquals(FAM1, ByteBufferUtils.toStringBinary(keyOnlyKv.getFamilyBuffer(), + keyOnlyKv.getFamilyOffset(), keyOnlyKv.getFamilyLength())); + assertEquals( + QUAL1, + ByteBufferUtils.toStringBinary(keyOnlyKv.getQualifierBuffer(), + keyOnlyKv.getQualifierOffset(), keyOnlyKv.getQualifierLength())); + assertEquals(0L, keyOnlyKv.getTimestamp()); + assertEquals(Type.Put.getCode(), keyOnlyKv.getTypeByte()); + } +} \ No newline at end of file diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/TestOnHeapBBBackedKeyValue.java hbase-common/src/test/java/org/apache/hadoop/hbase/TestOnHeapBBBackedKeyValue.java new file mode 100644 index 0000000..08d8c1d --- /dev/null +++ hbase-common/src/test/java/org/apache/hadoop/hbase/TestOnHeapBBBackedKeyValue.java @@ -0,0 +1,190 @@ +/** + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.KeyValue.MetaComparator; +import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class }) +public class TestOnHeapBBBackedKeyValue { + private static final String QUAL2 = "qual2"; + private static final String FAM2 = "fam2"; + private static final String QUAL1 = "qual1"; + private static final String FAM1 = "fam1"; + private static final String ROW1 = "row1"; + private static final ByteBuffer row = ByteBuffer.wrap(Bytes.toBytes(ROW1)); + private static final ByteBuffer family1 = (ByteBuffer.wrap(Bytes.toBytes(FAM1))); + private static final ByteBuffer qualifier1 = (ByteBuffer.wrap(Bytes.toBytes(QUAL1))); + private static final ByteBuffer family2 = (ByteBuffer.wrap(Bytes.toBytes(FAM2))); + private static final ByteBuffer qualifier2 = (ByteBuffer.wrap(Bytes.toBytes(QUAL2))); + private static final ArrayBackedTag t1 = new ArrayBackedTag((byte) 1, Bytes.toBytes("TAG1")); + private static final ArrayBackedTag t2 = new ArrayBackedTag((byte) 2, Bytes.toBytes("TAG2")); + private static final ArrayList tags = new ArrayList(); + static { + tags.add(t1); + tags.add(t2); + } + + @Test + public void testByteBufferBackedKeyValue() throws Exception { + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(row, family1, qualifier1, 0L, + Type.Put, row); + assertEquals(ROW1, + ByteBufferUtils.toStringBinary(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength())); + assertEquals( + FAM1, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + QUAL1, + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + assertEquals( + ROW1, + ByteBufferUtils.toStringBinary(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength())); + assertEquals(0L, kv.getTimestamp()); + assertEquals(Type.Put.getCode(), kv.getTypeByte()); + kv = new ByteBufferBackedKeyValue(row, family2, qualifier2, 0L, Type.Put, row); + assertEquals( + FAM2, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + QUAL2, + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + byte[] nullQualifier = new byte[0]; + kv = new ByteBufferBackedKeyValue(row, family1, (ByteBuffer.wrap(nullQualifier)), 0L, Type.Put, + row); + assertEquals(ROW1, + ByteBufferUtils.toStringBinary(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength())); + assertEquals( + FAM1, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + "", + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + assertEquals( + ROW1, + ByteBufferUtils.toStringBinary(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength())); + assertEquals(0L, kv.getTimestamp()); + assertEquals(Type.Put.getCode(), kv.getTypeByte()); + } + + @Test + public void testByteRangeBackedKeyValueWithTags() throws Exception { + + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(row, family1, qualifier1, 0L, + Type.Put, row, tags); + assertEquals(ROW1, + ByteBufferUtils.toStringBinary(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength())); + assertEquals( + FAM1, + ByteBufferUtils.toStringBinary(kv.getFamilyBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength())); + assertEquals( + QUAL1, + ByteBufferUtils.toStringBinary(kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength())); + assertEquals( + ROW1, + ByteBufferUtils.toStringBinary(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength())); + assertEquals(0L, kv.getTimestamp()); + assertEquals(Type.Put.getCode(), kv.getTypeByte()); + List resTags = TagUtil.asList(kv.getTagsBuffer(), kv.getTagsOffset(), kv.getTagsLength()); + Tag tag1 = resTags.get(0); + assertEquals(t1.getType(), tag1.getType()); + assertEquals(Bytes.toString(t1.getValue()), Bytes.toString(TagUtil.getTagValue(tag1))); + Tag tag2 = resTags.get(1); + assertEquals(tag2.getType(), tag2.getType()); + assertEquals(Bytes.toString(t2.getValue()), Bytes.toString(TagUtil.getTagValue(tag2))); + Tag res = TagUtil.getTag(kv.getTagsBuffer(), kv.getTagsOffset(), kv.getTagsLength(), (byte) 2); + assertEquals(Bytes.toString(t2.getValue()), Bytes.toString(TagUtil.getTagValue(tag2))); + res = TagUtil.getTag(kv.getTagsBuffer(), kv.getTagsOffset(), kv.getTagsLength(), (byte) 3); + assertNull(res); + } + + @Test + public void testGetKeyMethods() throws Exception { + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(row, family1, qualifier1, 0L, + Type.Put, row, tags); + ByteBuffer key = kv.getKey(); + assertEquals(key.capacity(), kv.getKeyLength()); + ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue keyOnlyKv = + new ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength()); + assertEquals(ROW1, ByteBufferUtils.toStringBinary(keyOnlyKv.getRowBuffer(), + keyOnlyKv.getRowOffset(), keyOnlyKv.getRowLength())); + assertEquals(FAM1, ByteBufferUtils.toStringBinary(keyOnlyKv.getFamilyBuffer(), + keyOnlyKv.getFamilyOffset(), keyOnlyKv.getFamilyLength())); + assertEquals( + QUAL1, + ByteBufferUtils.toStringBinary(keyOnlyKv.getQualifierBuffer(), + keyOnlyKv.getQualifierOffset(), keyOnlyKv.getQualifierLength())); + assertEquals(0L, keyOnlyKv.getTimestamp()); + assertEquals(Type.Put.getCode(), keyOnlyKv.getTypeByte()); + } + + @Test + public void testMetaKeyComparator() { + MetaComparator c = new KeyValue.MetaComparator(); + long now = System.currentTimeMillis(); + + Cell cell1 = new ByteBufferBackedKeyValue( + ByteBuffer + .wrap(Bytes + .toBytes("TestTable,00000000000000000000283916,1424107278653.56b74c32092ebf1f2d48e59497945209.")), + HConstants.EMPTY_BYTE_BUFFER, HConstants.EMPTY_BYTE_BUFFER, HConstants.LATEST_TIMESTAMP, + KeyValue.Type.Maximum, HConstants.EMPTY_BYTE_BUFFER); + ByteBufferBackedKeyValue cell2 = new ByteBufferBackedKeyValue(ByteBuffer.wrap(Bytes + .toBytes("TestTable,,1424107278653.56b74c32092ebf1f2d48e59497945209.")), + ByteBuffer.wrap(Bytes.toBytes("info")), ByteBuffer.wrap(Bytes.toBytes("regioninfo")), + 1424107279570l, KeyValue.Type.Put, HConstants.EMPTY_BYTE_BUFFER); + ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue k = + new ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue(); + k.setKey(cell2.getBuffer(), cell2.getKeyOffset(), cell2.getKeyLength()); + int res = c.compareRows(cell1, k); + assertEquals(26, res); + cell2 = new ByteBufferBackedKeyValue( + ByteBuffer + .wrap(Bytes + .toBytes("TestTable,00000000000000000000283916,1424107278653.56b74c32092ebf1f2d48e59497945209.")), + ByteBuffer.wrap(Bytes.toBytes("info")), ByteBuffer.wrap(Bytes.toBytes("regioninfo")), + 1424107279570l, KeyValue.Type.Put, HConstants.EMPTY_BYTE_BUFFER); + k.setKey(cell2.getBuffer(), cell2.getKeyOffset(), cell2.getKeyLength()); + res = c.compareRows(cell1, k); + assertEquals(0, res); + } +} \ No newline at end of file diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java index 30f2f00..eb34000 100644 --- hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java +++ hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java @@ -32,7 +32,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -54,16 +56,16 @@ public class TestCellCodecWithTags { Codec.Encoder encoder = codec.getEncoder(dos); final Cell cell1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("1"), new Tag[] { - new Tag((byte) 1, Bytes.toBytes("teststring1")), - new Tag((byte) 2, Bytes.toBytes("teststring2")) }); + new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring1")), + new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring2")) }); final Cell cell2 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("2"), - HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new Tag((byte) 1, + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring3")), }); final Cell cell3 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("3"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"), new Tag[] { - new Tag((byte) 2, Bytes.toBytes("teststring4")), - new Tag((byte) 2, Bytes.toBytes("teststring5")), - new Tag((byte) 1, Bytes.toBytes("teststring6")) }); + new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring4")), + new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring5")), + new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring6")) }); encoder.write(cell1); encoder.write(cell2); @@ -77,36 +79,36 @@ public class TestCellCodecWithTags { assertTrue(decoder.advance()); Cell c = decoder.current(); assertTrue(CellComparator.equals(c, cell1)); - List tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + List tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); assertEquals(2, tags.size()); Tag tag = tags.get(0); assertEquals(1, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), TagUtil.getTagValue(tag))); tag = tags.get(1); assertEquals(2, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), TagUtil.getTagValue(tag))); assertTrue(decoder.advance()); c = decoder.current(); assertTrue(CellComparator.equals(c, cell2)); - tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); assertEquals(1, tags.size()); tag = tags.get(0); assertEquals(1, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), TagUtil.getTagValue(tag))); assertTrue(decoder.advance()); c = decoder.current(); assertTrue(CellComparator.equals(c, cell3)); - tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); assertEquals(3, tags.size()); tag = tags.get(0); assertEquals(2, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), TagUtil.getTagValue(tag))); tag = tags.get(1); assertEquals(2, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), TagUtil.getTagValue(tag))); tag = tags.get(2); assertEquals(1, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), TagUtil.getTagValue(tag))); assertFalse(decoder.advance()); dis.close(); assertEquals(offset, cis.getCount()); diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java index 007647a..cf944ab 100644 --- hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java +++ hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java @@ -32,7 +32,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -54,16 +56,16 @@ public class TestKeyValueCodecWithTags { Codec.Encoder encoder = codec.getEncoder(dos); final KeyValue kv1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("1"), new Tag[] { - new Tag((byte) 1, Bytes.toBytes("teststring1")), - new Tag((byte) 2, Bytes.toBytes("teststring2")) }); + new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring1")), + new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring2")) }); final KeyValue kv2 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("2"), - HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new Tag((byte) 1, + HConstants.LATEST_TIMESTAMP, Bytes.toBytes("2"), new Tag[] { new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring3")), }); final KeyValue kv3 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("3"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"), new Tag[] { - new Tag((byte) 2, Bytes.toBytes("teststring4")), - new Tag((byte) 2, Bytes.toBytes("teststring5")), - new Tag((byte) 1, Bytes.toBytes("teststring6")) }); + new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring4")), + new ArrayBackedTag((byte) 2, Bytes.toBytes("teststring5")), + new ArrayBackedTag((byte) 1, Bytes.toBytes("teststring6")) }); encoder.write(kv1); encoder.write(kv2); @@ -77,36 +79,36 @@ public class TestKeyValueCodecWithTags { assertTrue(decoder.advance()); Cell c = decoder.current(); assertTrue(CellComparator.equals(c, kv1)); - List tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + List tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); assertEquals(2, tags.size()); Tag tag = tags.get(0); assertEquals(1, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), TagUtil.getTagValue(tag))); tag = tags.get(1); assertEquals(2, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), TagUtil.getTagValue(tag))); assertTrue(decoder.advance()); c = decoder.current(); assertTrue(CellComparator.equals(c, kv2)); - tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); assertEquals(1, tags.size()); tag = tags.get(0); assertEquals(1, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), TagUtil.getTagValue(tag))); assertTrue(decoder.advance()); c = decoder.current(); assertTrue(CellComparator.equals(c, kv3)); - tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); + tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength()); assertEquals(3, tags.size()); tag = tags.get(0); assertEquals(2, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), TagUtil.getTagValue(tag))); tag = tags.get(1); assertEquals(2, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), TagUtil.getTagValue(tag))); tag = tags.get(2); assertEquals(1, tag.getType()); - assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), tag.getValue())); + assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), TagUtil.getTagValue(tag))); assertFalse(decoder.advance()); dis.close(); assertEquals(offset, cis.getCount()); diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java index 841c468..99c01e9 100644 --- hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java +++ hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java @@ -22,11 +22,11 @@ import static org.junit.Assert.assertTrue; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.util.LRUDictionary; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -44,32 +44,6 @@ public class TestTagCompressionContext { private static final byte[] V = Bytes.toBytes("v"); @Test - public void testCompressUncompressTags1() throws Exception { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE); - KeyValue kv1 = createKVWithTags(2); - int tagsLength1 = kv1.getTagsLength(); - ByteBuffer ib = ByteBuffer.wrap(kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1); - context.compressTags(baos, ib, tagsLength1); - KeyValue kv2 = createKVWithTags(3); - int tagsLength2 = kv2.getTagsLength(); - ib = ByteBuffer.wrap(kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2); - context.compressTags(baos, ib, tagsLength2); - - context.clear(); - - byte[] dest = new byte[tagsLength1]; - ByteBuffer ob = ByteBuffer.wrap(baos.toByteArray()); - context.uncompressTags(ob, dest, 0, tagsLength1); - assertTrue(Bytes.equals(kv1.getTagsArray(), kv1.getTagsOffset(), tagsLength1, dest, 0, - tagsLength1)); - dest = new byte[tagsLength2]; - context.uncompressTags(ob, dest, 0, tagsLength2); - assertTrue(Bytes.equals(kv2.getTagsArray(), kv2.getTagsOffset(), tagsLength2, dest, 0, - tagsLength2)); - } - - @Test public void testCompressUncompressTags2() throws Exception { ByteArrayOutputStream baos = new ByteArrayOutputStream(); TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE); @@ -96,7 +70,7 @@ public class TestTagCompressionContext { private KeyValue createKVWithTags(int noOfTags) { List tags = new ArrayList(); for (int i = 0; i < noOfTags; i++) { - tags.add(new Tag((byte) i, "tagValue" + i)); + tags.add(new ArrayBackedTag((byte) i, "tagValue" + i)); } KeyValue kv = new KeyValue(ROW, CF, Q, 1234L, V, tags); return kv; diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestMultiByteBuffer.java hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestMultiByteBuffer.java new file mode 100644 index 0000000..699413a --- /dev/null +++ hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestMultiByteBuffer.java @@ -0,0 +1,185 @@ +/** + * Copyright The Apache Software Foundation + * + * 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.io.util; + +import static org.junit.Assert.assertEquals; + +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MiscTests.class, SmallTests.class }) +public class TestMultiByteBuffer { + + @Test + public void testAbsoluteReads() { + ByteBuffer pbr1 = ByteBuffer.allocate(15); + ByteBuffer pbr2 = ByteBuffer.allocate(15); + pbr1.putInt(4); + long l1= 45L, l2 = 100L, l3 = 12345L; + pbr1.putLong(l1); + pbr1.putShort((short) 2); + byte[] b = Bytes.toBytes(l2); + pbr1.put(b, 0, 1); + pbr2.put(b, 1, 7); + pbr2.putLong(l3); + MultiByteBuffer multi = new MultiByteBuffer(pbr1, pbr2); + assertEquals(l1, multi.getLong(4)); + assertEquals(l2, multi.getLong(14)); + assertEquals(l3, multi.getLong(22)); + } + + @Test + public void testMarkAndReset() { + ByteBuffer pbr1 = ByteBuffer.allocate(15); + ByteBuffer pbr2 = ByteBuffer.allocate(15); + pbr1.putInt(4); + long l1= 45L, l2 = 100L, l3 = 12345L; + pbr1.putLong(l1); + pbr1.putShort((short) 2); + byte[] b = Bytes.toBytes(l2); + pbr1.put(b, 0, 1); + pbr2.put(b, 1, 7); + //pbr2.putLong(l2); + pbr2.putLong(l3); + MultiByteBuffer multi = new MultiByteBuffer(pbr1, pbr2); + assertEquals(4, multi.getInt()); + assertEquals(l1, multi.getLong()); + multi.mark(); + assertEquals((short)2, multi.getShort()); + multi.reset(); + assertEquals((short)2, multi.getShort()); + multi.mark(); + assertEquals(l2, multi.getLong()); + multi.reset(); + assertEquals(l2, multi.getLong()); + multi.mark(); + assertEquals(l3, multi.getLong()); + multi.reset(); + assertEquals(l3, multi.getLong()); + // Try absolute gets with mark and reset + multi.mark(); + assertEquals(l2, multi.getLong(14)); + multi.reset(); + assertEquals(l3, multi.getLong(22)); + //Just reset to see what happens + multi.reset(); + assertEquals(l2, multi.getLong(14)); + multi.mark(); + assertEquals(l3, multi.getLong(22)); + multi.reset(); + } + @Test + public void testSkipNBytes() { + ByteBuffer pbr1 = ByteBuffer.allocate(15); + ByteBuffer pbr2 = ByteBuffer.allocate(15); + pbr1.putInt(4); + long l1= 45L, l2 = 100L, l3 = 12345L; + pbr1.putLong(l1); + pbr1.putShort((short) 2); + byte[] b = Bytes.toBytes(l2); + pbr1.put(b, 0, 1); + pbr2.put(b, 1, 7); + //pbr2.putLong(l2); + pbr2.putLong(l3); + MultiByteBuffer multi = new MultiByteBuffer(pbr1, pbr2); + assertEquals(4, multi.getInt()); + assertEquals(l1, multi.getLong()); + multi.skip(10); + assertEquals(l3, multi.getLong()); + } + + @Test + public void testRewind() { + ByteBuffer pbr1 = ByteBuffer.allocate(15); + ByteBuffer pbr2 = ByteBuffer.allocate(15); + pbr1.putInt(4); + long l1= 45L, l2 = 100L, l3 = 12345L; + pbr1.putLong(l1); + pbr1.putShort((short) 2); + byte[] b = Bytes.toBytes(l2); + pbr1.put(b, 0, 1); + pbr2.put(b, 1, 7); + //pbr2.putLong(l2); + pbr2.putLong(l3); + MultiByteBuffer multi = new MultiByteBuffer(pbr1, pbr2); + assertEquals(4, multi.getInt()); + assertEquals(l1, multi.getLong()); + multi.skip(10); + assertEquals(l3, multi.getLong()); + multi.rewind(); + assertEquals(4, multi.getInt()); + assertEquals(l1, multi.getLong()); + multi.skip(10); + assertEquals(l3, multi.getLong()); + } + + @Test + public void testMoveBack() { + ByteBuffer pbr1 = ByteBuffer.allocate(15); + ByteBuffer pbr2 = ByteBuffer.allocate(15); + pbr1.putInt(4); + long l1= 45L, l2 = 100L, l3 = 12345L; + pbr1.putLong(l1); + pbr1.putShort((short) 2); + byte[] b = Bytes.toBytes(l2); + pbr1.put(b, 0, 1); + pbr2.put(b, 1, 7); + //pbr2.putLong(l2); + pbr2.putLong(l3); + MultiByteBuffer multi = new MultiByteBuffer(pbr1, pbr2); + assertEquals(4, multi.getInt()); + assertEquals(l1, multi.getLong()); + multi.skip(10); + multi.moveBack(4); + multi.moveBack(6); + multi.moveBack(8); + assertEquals(l1, multi.getLong()); + } + + @Test + public void asSubBuffer() { + ByteBuffer pbr1 = ByteBuffer.allocate(15); + ByteBuffer pbr2 = ByteBuffer.allocate(15); + pbr1.putInt(4); + long l1= 45L, l2 = 100L, l3 = 12345L; + pbr1.putLong(l1); + pbr1.putShort((short) 2); + byte[] b = Bytes.toBytes(l2); + pbr1.put(b, 0, 1); + pbr2.put(b, 1, 7); + //pbr2.putLong(l2); + pbr2.putLong(l3); + MultiByteBuffer multi = new MultiByteBuffer(pbr1, pbr2); + assertEquals(4, multi.getInt()); + assertEquals(l1, multi.getLong()); + assertEquals(2, multi.getShort()); + assertEquals(l2, multi.getLong()); + multi.rewind(); + Pair p = multi.asSubBuffer(12, 2); + assertEquals((short)2, p.getFirst().getShort(p.getSecond())); + } +} diff --git hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java index a6b7cc5..ecdd220 100644 --- hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java +++ hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteRangeWithKVSerialization.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -65,7 +66,7 @@ public class TestByteRangeWithKVSerialization { int kvCount = 1000000; List kvs = new ArrayList(kvCount); int totalSize = 0; - Tag[] tags = new Tag[] { new Tag((byte) 1, "tag1") }; + Tag[] tags = new ArrayBackedTag[] { new ArrayBackedTag((byte) 1, "tag1") }; for (int i = 0; i < kvCount; i++) { KeyValue kv = new KeyValue(Bytes.toBytes(i), FAMILY, QUALIFIER, i, VALUE, tags); kv.setSequenceId(i); diff --git hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java index afcb526..8514f1d 100644 --- hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java +++ hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java @@ -23,13 +23,13 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; import org.apache.hadoop.hbase.KeyValue.MetaComparator; import org.apache.hadoop.hbase.KeyValue.RawBytesComparator; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory; import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher; import org.apache.hadoop.hbase.codec.prefixtree.encode.EncoderFactory; @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.io.WritableUtils; /** @@ -114,12 +115,12 @@ public class PrefixTreeCodec implements DataBlockEncoder{ @Override - public ByteBuffer getFirstKeyInBlock(ByteBuffer block) { + public ByteBuffer getFirstKeyInBlock(MultiByteBuffer block) { block.rewind(); PrefixTreeArraySearcher searcher = null; try { // should i includeMemstoreTS (second argument)? i think PrefixKeyDeltaEncoder is, so i will - searcher = DecoderFactory.checkOut(block, true); + searcher = DecoderFactory.checkOut(block.asSubBuffer(block.limit() - block.position()), true); if (!searcher.positionAtFirstCell()) { return null; } diff --git hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java index b95055c..6e83391 100644 --- hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java +++ hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher; import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; @@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.util.ClassSize; @InterfaceAudience.Private public class PrefixTreeSeeker implements EncodedSeeker { + // This should be now MBB. // TODO - change it. Will be a big change protected ByteBuffer block; protected boolean includeMvccVersion; protected PrefixTreeArraySearcher ptSearcher; @@ -57,6 +59,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override public void setCurrentBuffer(ByteBuffer fullBlockBuffer) { + // TODO : HAck to avoid compilation errors block = fullBlockBuffer; ptSearcher = DecoderFactory.checkOut(block, includeMvccVersion); rewind(); @@ -75,13 +78,14 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override public ByteBuffer getKeyDeepCopy() { + // TODO : hack - but this can be local onheap return KeyValueUtil.copyKeyToNewByteBuffer(ptSearcher.current()); } @Override public ByteBuffer getValueShallowCopy() { - return CellUtil.getValueBufferShallowCopy(ptSearcher.current()); + return (CellUtil.getValueBufferShallowCopy(ptSearcher.current())); } /** @@ -89,7 +93,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { */ @Override public ByteBuffer getKeyValueBuffer() { - return KeyValueUtil.copyToNewByteBuffer(ptSearcher.current()); + return (KeyValueUtil.copyToNewByteBuffer(ptSearcher.current())); } /** @@ -101,10 +105,10 @@ public class PrefixTreeSeeker implements EncodedSeeker { if (cell == null) { return null; } - return new ClonedPrefixTreeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), - cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), - cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), - cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(), cell.getTagsArray(), + return new ClonedPrefixTreeCell(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength(), + cell.getFamilyBuffer(), cell.getFamilyOffset(), cell.getFamilyLength(), + cell.getQualifierBuffer(), cell.getQualifierOffset(), cell.getQualifierLength(), + cell.getValueBuffer(), cell.getValueOffset(), cell.getValueLength(), cell.getTagsBuffer(), cell.getTagsOffset(), cell.getTagsLength(), cell.getTimestamp(), cell.getTypeByte(), cell.getSequenceId()); } @@ -245,7 +249,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { public int compareKey(KVComparator comparator, byte[] key, int offset, int length) { // can't optimize this, make a copy of the key ByteBuffer bb = getKeyDeepCopy(); - return comparator.compareFlatKey(key, offset, length, bb.array(), bb.arrayOffset(), bb.limit()); + return comparator.compareFlatKey(key, offset, length, bb, bb.arrayOffset(), bb.limit()); } @Override @@ -272,36 +276,36 @@ public class PrefixTreeSeeker implements EncodedSeeker { private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (5 * ClassSize.ARRAY)); - private byte[] row; + private ByteBuffer row; private short rowLength; - private byte[] fam; + private ByteBuffer fam; private byte famLength; - private byte[] qual; + private ByteBuffer qual; private int qualLength; - private byte[] val; + private ByteBuffer val; private int valOffset; private int valLength; - private byte[] tag; + private ByteBuffer tag; private int tagsLength; private long ts; private long seqId; private byte type; - public ClonedPrefixTreeCell(byte[] row, int rowOffset, short rowLength, byte[] fam, - int famOffset, byte famLength, byte[] qual, int qualOffset, int qualLength, byte[] val, - int valOffset, int valLength, byte[] tag, int tagOffset, int tagLength, long ts, byte type, + public ClonedPrefixTreeCell(ByteBuffer row, int rowOffset, short rowLength, ByteBuffer fam, + int famOffset, byte famLength, ByteBuffer qual, int qualOffset, int qualLength, ByteBuffer val, + int valOffset, int valLength, ByteBuffer tag, int tagOffset, int tagLength, long ts, byte type, long seqId) { - this.row = new byte[rowLength]; - System.arraycopy(row, rowOffset, this.row, 0, rowLength); + this.row = ByteBuffer.allocate(rowLength); + ByteBufferUtils.copyFromBufferToBuffer(this.row, row, rowOffset, rowLength); this.rowLength = rowLength; - this.fam = new byte[famLength]; - System.arraycopy(fam, famOffset, this.fam, 0, famLength); + this.fam = ByteBuffer.allocate(famLength); + ByteBufferUtils.copyFromBufferToBuffer(this.fam, fam, famOffset, famLength); this.famLength = famLength; - this.qual = new byte[qualLength]; - System.arraycopy(qual, qualOffset, this.qual, 0, qualLength); + this.qual = ByteBuffer.allocate(qualLength); + ByteBufferUtils.copyFromBufferToBuffer(this.qual, qual, qualOffset, qualLength); this.qualLength = qualLength; - this.tag = new byte[tagLength]; - System.arraycopy(tag, tagOffset, this.tag, 0, tagLength); + this.tag = ByteBuffer.allocate(tagLength); + ByteBufferUtils.copyFromBufferToBuffer(this.tag, tag, tagOffset, tagLength); this.tagsLength = tagLength; this.val = val; this.valLength = valLength; @@ -317,8 +321,12 @@ public class PrefixTreeSeeker implements EncodedSeeker { } @Override + public boolean hasArray() { + return false; + } + @Override public byte[] getRowArray() { - return this.row; + return this.row.array(); } @Override @@ -333,7 +341,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override public byte[] getFamilyArray() { - return this.fam; + return this.fam.array(); } @Override @@ -348,7 +356,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override public byte[] getQualifierArray() { - return this.qual; + return this.qual.array(); } @Override @@ -384,7 +392,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override public byte[] getValueArray() { - return val; + return val.array(); } @Override @@ -399,7 +407,7 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override public byte[] getTagsArray() { - return this.tag; + return this.tag.array(); } @Override @@ -415,25 +423,25 @@ public class PrefixTreeSeeker implements EncodedSeeker { @Override @Deprecated public byte[] getValue() { - return this.val; + return this.val.array(); } @Override @Deprecated public byte[] getFamily() { - return this.fam; + return this.fam.array(); } @Override @Deprecated public byte[] getQualifier() { - return this.qual; + return this.qual.array(); } @Override @Deprecated public byte[] getRow() { - return this.row; + return this.row.array(); } @Override @@ -451,5 +459,35 @@ public class PrefixTreeSeeker implements EncodedSeeker { public long heapSize() { return FIXED_OVERHEAD + rowLength + famLength + qualLength + valLength + tagsLength; } + + @Override + public ByteBuffer getRowBuffer() { + // TODO Auto-generated method stub + return row; + } + + @Override + public ByteBuffer getFamilyBuffer() { + // TODO Auto-generated method stub + return fam; + } + + @Override + public ByteBuffer getQualifierBuffer() { + // TODO Auto-generated method stub + return qual; + } + + @Override + public ByteBuffer getValueBuffer() { + // TODO Auto-generated method stub + return val; + } + + @Override + public ByteBuffer getTagsBuffer() { + // TODO Auto-generated method stub + return tag; + } } } diff --git hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java index 97eed62..764e680 100644 --- hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java +++ hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.codec.prefixtree.decode; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; @@ -25,11 +27,14 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.SettableSequenceId; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.ByteRange; /** * As the PrefixTreeArrayScanner moves through the tree bytes, it changes the values in the fields * of this class so that Cell logic can be applied, but without allocating new memory for every Cell * iterated through. + * We make all the getXXXOffset as 0 and while returning getXXXBuffer would wrap the bytes in a bytebuffer. + * hasArray is false here */ @InterfaceAudience.Private public class PrefixTreeCell implements Cell, SettableSequenceId, Comparable { @@ -112,6 +117,10 @@ public class PrefixTreeCell implements Cell, SettableSequenceId, Comparable d = Lists.newArrayList(); static { List tagList = new ArrayList(); - Tag t = new Tag((byte) 1, "visisbility"); + Tag t = new ArrayBackedTag((byte) 1, "visisbility"); tagList.add(t); - t = new Tag((byte) 2, "ACL"); + t = new ArrayBackedTag((byte) 2, "ACL"); tagList.add(t); d.add(new KeyValue(rA, cf, cq0, ts, v0, tagList)); d.add(new KeyValue(rB, cf, cq0, ts, v0, tagList)); diff --git hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java index e91f873..2f0e1c7 100644 --- hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java +++ hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java @@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -1124,7 +1125,7 @@ public class PerformanceEvaluation extends Configured implements Tool { byte[] tag = generateData(this.rand, TAG_LENGTH); Tag[] tags = new Tag[noOfTags]; for (int n = 0; n < noOfTags; n++) { - Tag t = new Tag((byte) n, tag); + Tag t = new ArrayBackedTag((byte) n, tag); tags[n] = t; } KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP, @@ -1195,7 +1196,7 @@ public class PerformanceEvaluation extends Configured implements Tool { byte[] tag = generateData(this.rand, TAG_LENGTH); Tag[] tags = new Tag[noOfTags]; for (int n = 0; n < noOfTags; n++) { - Tag t = new Tag((byte) n, tag); + Tag t = new ArrayBackedTag((byte) n, tag); tags[n] = t; } KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/TagRewriteCell.java hbase-server/src/main/java/org/apache/hadoop/hbase/TagRewriteCell.java index 313ecb8..779e70c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/TagRewriteCell.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/TagRewriteCell.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; @@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.util.ClassSize; */ @InterfaceAudience.Private public class TagRewriteCell implements Cell, SettableSequenceId, SettableTimestamp, HeapSize { - private Cell cell; private byte[] tags; @@ -49,6 +49,11 @@ public class TagRewriteCell implements Cell, SettableSequenceId, SettableTimesta ((TagRewriteCell) this.cell).tags = null; } } + + @Override + public boolean hasArray() { + return cell.hasArray(); + } @Override public byte[] getRowArray() { @@ -199,4 +204,28 @@ public class TagRewriteCell implements Cell, SettableSequenceId, SettableTimesta // The incoming cell is supposed to be SettableSequenceId type. CellUtil.setSequenceId(cell, seqId); } + + public ByteBuffer getRowBuffer() { + return cell.getRowBuffer(); + } + + @Override + public ByteBuffer getFamilyBuffer() { + return cell.getFamilyBuffer(); + } + + @Override + public ByteBuffer getQualifierBuffer() { + return cell.getQualifierBuffer(); + } + + @Override + public ByteBuffer getValueBuffer() { + return cell.getValueBuffer(); + } + + @Override + public ByteBuffer getTagsBuffer() { + return ByteBuffer.wrap(this.tags); + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java index 6c894a5..a9f759a 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java @@ -21,12 +21,12 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import org.apache.hadoop.hbase.util.ByteStringer; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.protobuf.generated.CellProtos; +import org.apache.hadoop.hbase.util.ByteStringer; /** * Codec that just writes out Cell as a protobuf Cell Message. Does not write the mvcc stamp. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java index 215ff16..2929404 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java @@ -16,6 +16,7 @@ package org.apache.hadoop.hbase.coprocessor; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import java.util.NavigableSet; @@ -434,6 +435,13 @@ public abstract class BaseRegionObserver implements RegionObserver { final boolean hasMore) throws IOException { return hasMore; } + + @Override + public boolean postScannerFilterRow(ObserverContext c, + InternalScanner s, ByteBuffer currentRow, int offset, short length, boolean hasMore) + throws IOException { + return hasMore; + } @Override public void preScannerClose(final ObserverContext e, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 9fede52..f775a4b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -16,6 +16,7 @@ package org.apache.hadoop.hbase.coprocessor; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import java.util.NavigableSet; @@ -1079,11 +1080,36 @@ public interface RegionObserver extends Coprocessor { * @return whether more rows are available for the scanner or not * @throws IOException */ + @Deprecated boolean postScannerFilterRow(final ObserverContext c, final InternalScanner s, final byte[] currentRow, final int offset, final short length, final boolean hasMore) throws IOException; /** + * This will be called by the scan flow when the current scanned row is being filtered out by the + * filter. The filter may be filtering out the row via any of the below scenarios + *
    + *
  1. + * boolean filterRowKey(byte [] buffer, int offset, int length) returning true
  2. + *
  3. + * boolean filterRow() returning true
  4. + *
  5. + * void filterRow(List kvs) removing all the kvs from the passed List
  6. + *
+ * @param c the environment provided by the region server + * @param s the scanner + * @param currentRow The current rowkey which got filtered out + * @param offset offset to rowkey + * @param length length of rowkey + * @param hasMore the 'has more' indication + * @return whether more rows are available for the scanner or not + * @throws IOException + */ + boolean postScannerFilterRow(final ObserverContext c, + final InternalScanner s, final ByteBuffer currentRow, final int offset, final short length, + final boolean hasMore) throws IOException; + + /** * Called before the client closes a scanner. *

* Call CoprocessorEnvironment#bypass to skip default actions diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java index 43bbab5..cd18a8f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * A facade for a {@link org.apache.hadoop.hbase.io.hfile.HFile.Reader} that serves up @@ -164,8 +166,7 @@ public class HalfStoreFileReader extends StoreFile.Reader { // constrain the bottom. if (!top) { ByteBuffer bb = getKey(); - if (getComparator().compareFlatKey(bb.array(), bb.arrayOffset(), bb.limit(), - splitkey, 0, splitkey.length) >= 0) { + if (getComparator().compareFlatKey(bb, 0, bb.limit(), splitkey, 0, splitkey.length) >= 0) { atEnd = true; return false; } @@ -207,9 +208,13 @@ public class HalfStoreFileReader extends StoreFile.Reader { } // Check key. ByteBuffer k = this.delegate.getKey(); - return this.delegate.getReader().getComparator(). - compareFlatKey(k.array(), k.arrayOffset(), k.limit(), - splitkey, 0, splitkey.length) < 0; + return this.delegate.getReader().getComparator() + .compareFlatKey(k, 0, k.limit(), splitkey, 0, splitkey.length) < 0; + } + + @Override + public void close() { + delegate.close(); } @Override diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java index 8c1e7b9..62e39f4 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configurable; @@ -32,6 +31,8 @@ import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; +import org.apache.hadoop.hbase.io.hfile.HFileBlock.CacheType; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Common functionality needed by all versions of {@link HFile} readers. @@ -266,8 +267,18 @@ public abstract class AbstractHFileReader } } + @Override + public void returnBlock(HFileBlock block) { + BlockCache blockCache = this.cacheConf.getBlockCache(); + if (blockCache != null && block != null && block.getCacheType() != CacheType.NOT_CACHED) { + BlockCacheKey cacheKey = new BlockCacheKey(this.getFileContext().getHFileName(), + block.getOffset()); + blockCache.returnBlock(cacheKey, block); + } + } + protected static abstract class Scanner implements HFileScanner { - protected ByteBuffer blockBuffer; + protected MultiByteBuffer blockBuffer; protected boolean cacheBlocks; protected final boolean pread; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java index 52491e6..1fa6f10 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java @@ -211,10 +211,15 @@ public abstract class AbstractHFileWriter implements HFile.Writer { } /** Checks the given value for validity. */ - protected void checkValue(final byte[] value, final int offset, - final int length) throws IOException { - if (value == null) { - throw new IOException("Value cannot be null"); + protected void checkValue(final Cell cell) throws IOException { + if(cell.hasArray()) { + if(cell.getValueArray() == null) { + throw new IOException("Value cannot be null"); + } + } else { + if(cell.getValueBuffer() == null) { + throw new IOException("Value cannot be null"); + } } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java index 57c4be9..cc425e1 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java @@ -116,4 +116,9 @@ public interface BlockCache extends Iterable { * @return The list of sub blockcaches that make up this one; returns null if no sub caches. */ BlockCache [] getBlockCaches(); + + // TODO : Check if HFileBlock is needed here + // Call when this block usage is over. + boolean returnBlock(BlockCacheKey cacheKey, HFileBlock block); + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializer.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializer.java index f56a921..88909e4 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializer.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheableDeserializer.java @@ -18,9 +18,9 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * Interface for a deserializer. Throws an IOException if the serialized data is @@ -28,12 +28,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; * */ @InterfaceAudience.Private public interface CacheableDeserializer { - /** - * Returns the deserialized object. - * - * @return T the deserialized object. - */ - T deserialize(ByteBuffer b) throws IOException; /** * @@ -43,7 +37,7 @@ public interface CacheableDeserializer { * @return T the deserialized object. * @throws IOException */ - T deserialize(ByteBuffer b, boolean reuse) throws IOException; + T deserialize(MultiByteBuffer b, boolean reuse) throws IOException; /** * Get the identifier of this deserialiser. Identifier is unique for each diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java index 0e03a42..c7e6ffa 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java @@ -24,7 +24,6 @@ import java.util.zip.Checksum; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; @@ -72,7 +71,6 @@ public class ChecksumUtil { Checksum checksum = checksumType.getChecksumObject(); int bytesLeft = endOffset - startOffset; - int chunkNum = 0; while (bytesLeft > 0) { // generate the checksum for one chunk @@ -83,7 +81,6 @@ public class ChecksumUtil { // write the checksum value to the output buffer. int cksumValue = (int)checksum.getValue(); outOffset = Bytes.putInt(outdata, outOffset, cksumValue); - chunkNum++; startOffset += count; bytesLeft -= count; } @@ -134,11 +131,13 @@ public class ChecksumUtil { return false; // cannot happen case, unable to verify checksum } // Extract the header and compute checksum for the header. - ByteBuffer hdr = block.getBufferWithHeader(); + ByteBuffer hdr = block.getBufferWithHeader().asSubBuffer(hdrSize); if (hdr.hasArray()) { checksumObject.update(hdr.array(), hdr.arrayOffset(), hdrSize); } else { - checksumObject.update(ByteBufferUtils.toBytes(hdr, 0, hdrSize), 0, hdrSize); + byte[] output = new byte[hdrSize]; + hdr.get(output); + checksumObject.update(output, 0, hdrSize); } int off = hdrSize; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java index 52a5793..2112de8 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java @@ -23,6 +23,7 @@ import java.util.Iterator; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; +import org.apache.hadoop.hbase.io.hfile.HFileBlock.CacheType; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; /** @@ -212,4 +213,18 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize { public void setMaxSize(long size) { this.lruCache.setMaxSize(size); } + + + @Override + public boolean returnBlock(BlockCacheKey cacheKey, HFileBlock block) { + assert block.getCacheType() != CacheType.NOT_CACHED; + if (block.getCacheType() == CacheType.L1_CACHED) { + // At the time when this block was served, it was in L1 cache. Even if it is transferred to L2 + // cache by this time, no need to contact L2 cache. + return this.lruCache.returnBlock(cacheKey, block); + } else if (block.getCacheType() == CacheType.L2_CACHED) { + return this.bucketCache.returnBlock(cacheKey, block); + } + return false; + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 610fe7f..aaff812 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -26,7 +26,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.SequenceInputStream; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; @@ -39,7 +38,6 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -51,6 +49,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.compress.Compression; @@ -370,6 +369,11 @@ public class HFile { final boolean updateCacheMetrics, BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding) throws IOException; + /** + * Return the given block back to the cache, if it was obtained from cache. + * @param block Block to be returned. + */ + void returnBlock(HFileBlock block); } /** An interface used by clients to open and iterate an {@link HFile}. */ @@ -385,7 +389,7 @@ public class HFile { HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction); - ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException; + HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException; Map loadFileInfo() throws IOException; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java index b096185..b28569f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -34,8 +34,8 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.fs.HFileSystem; -import org.apache.hadoop.hbase.io.ByteBufferInputStream; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; +import org.apache.hadoop.hbase.io.MultiByteBufferInputStream; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext; import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext; @@ -45,6 +45,12 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ChecksumType; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.CompoundBloomFilter; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.io.IOUtils; import com.google.common.base.Preconditions; @@ -116,16 +122,22 @@ public class HFileBlock implements Cacheable { */ static final int CHECKSUM_SIZE = Bytes.SIZEOF_INT; + private CacheType cacheType = CacheType.NOT_CACHED; + private static final CacheableDeserializer blockDeserializer = new CacheableDeserializer() { - public HFileBlock deserialize(ByteBuffer buf, boolean reuse) throws IOException{ + + @Override + public Cacheable deserialize(MultiByteBuffer buf, boolean reuse) throws IOException { buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind(); - ByteBuffer newByteBuffer; + MultiByteBuffer newByteBuffer; if (reuse) { newByteBuffer = buf.slice(); } else { - newByteBuffer = ByteBuffer.allocate(buf.limit()); - newByteBuffer.put(buf); + int len = buf.limit(); + ByteBuffer bb = ByteBuffer.allocate(len); + MultiByteBuffer.copyFromBufferToBuffer(bb, buf, 0, len); + newByteBuffer = new MultiByteBuffer(bb); } buf.position(buf.limit()); buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE); @@ -143,11 +155,6 @@ public class HFileBlock implements Cacheable { public int getDeserialiserIdentifier() { return deserializerIdentifier; } - - @Override - public HFileBlock deserialize(ByteBuffer b) throws IOException { - return deserialize(b, false); - } }; private static final int deserializerIdentifier; static { @@ -174,7 +181,7 @@ public class HFileBlock implements Cacheable { private final int onDiskDataSizeWithHeader; /** The in-memory representation of the hfile block */ - private ByteBuffer buf; + private MultiByteBuffer buf; /** Meta data that holds meta information on the hfileblock */ private HFileContext fileContext; @@ -209,7 +216,7 @@ public class HFileBlock implements Cacheable { * @param fileContext HFile meta data */ HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, - long prevBlockOffset, ByteBuffer buf, boolean fillHeader, long offset, + long prevBlockOffset, MultiByteBuffer buf, boolean fillHeader, long offset, int onDiskDataSizeWithHeader, HFileContext fileContext) { this.blockType = blockType; this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader; @@ -224,6 +231,13 @@ public class HFileBlock implements Cacheable { this.buf.rewind(); } + HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader, + long prevBlockOffset, ByteBuffer buf, boolean fillHeader, long offset, + int onDiskDataSizeWithHeader, HFileContext fileContext) { + this(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, prevBlockOffset, + new MultiByteBuffer(buf), fillHeader, offset, onDiskDataSizeWithHeader, fileContext); + } + /** * Copy constructor. Creates a shallow copy of {@code that}'s buffer. */ @@ -239,6 +253,10 @@ public class HFileBlock implements Cacheable { this.nextBlockOnDiskSizeWithHeader = that.nextBlockOnDiskSizeWithHeader; } + HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException { + this(new MultiByteBuffer(b), usesHBaseChecksum); + } + /** * Creates a block from an existing buffer starting with a header. Rewinds * and takes ownership of the buffer. By definition of rewind, ignores the @@ -247,7 +265,8 @@ public class HFileBlock implements Cacheable { * because majorNumbers indicate the format of a HFile whereas minorNumbers * indicate the format inside a HFileBlock. */ - HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException { + HFileBlock(MultiByteBuffer b, boolean usesHBaseChecksum) + throws IOException { b.rewind(); blockType = BlockType.read(b); onDiskSizeWithoutHeader = b.getInt(); @@ -334,8 +353,8 @@ public class HFileBlock implements Cacheable { * * @return the buffer with header skipped and checksum omitted. */ - public ByteBuffer getBufferWithoutHeader() { - ByteBuffer dup = this.buf.duplicate(); + public MultiByteBuffer getBufferWithoutHeader() { + MultiByteBuffer dup = this.buf.duplicate(); dup.position(headerSize()); dup.limit(buf.limit() - totalChecksumBytes()); return dup.slice(); @@ -350,8 +369,8 @@ public class HFileBlock implements Cacheable { * * @return the buffer of this block for read-only operations */ - public ByteBuffer getBufferReadOnly() { - ByteBuffer dup = this.buf.duplicate(); + public MultiByteBuffer getBufferReadOnly() { + MultiByteBuffer dup = this.buf.duplicate(); dup.limit(buf.limit() - totalChecksumBytes()); return dup.slice(); } @@ -363,8 +382,8 @@ public class HFileBlock implements Cacheable { * * @return the buffer with header and checksum included for read-only operations */ - public ByteBuffer getBufferReadOnlyWithHeader() { - ByteBuffer dup = this.buf.duplicate(); + public MultiByteBuffer getBufferReadOnlyWithHeader() { + MultiByteBuffer dup = this.buf.duplicate(); return dup.slice(); } @@ -374,8 +393,8 @@ public class HFileBlock implements Cacheable { * * @return the byte buffer with header and checksum included */ - ByteBuffer getBufferWithHeader() { - ByteBuffer dupBuf = buf.duplicate(); + MultiByteBuffer getBufferWithHeader() { + MultiByteBuffer dupBuf = buf.duplicate(); dupBuf.rewind(); return dupBuf; } @@ -463,7 +482,7 @@ public class HFileBlock implements Cacheable { dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset() + headerSize(), Math.min(32, buf.limit() - buf.arrayOffset() - headerSize())); } else { - ByteBuffer bufWithoutHeader = getBufferWithoutHeader(); + MultiByteBuffer bufWithoutHeader = getBufferWithoutHeader(); byte[] dataBeginBytes = new byte[Math.min(32, bufWithoutHeader.limit() - bufWithoutHeader.position())]; bufWithoutHeader.get(dataBeginBytes); @@ -487,9 +506,10 @@ public class HFileBlock implements Cacheable { if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) { String dataBegin = null; if (buf.hasArray()) { - dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset(), Math.min(32, buf.limit())); + dataBegin = Bytes.toStringBinary(buf.array(), buf.arrayOffset(), + Math.min(32, buf.limit())); } else { - ByteBuffer bufDup = getBufferReadOnly(); + MultiByteBuffer bufDup = getBufferReadOnly(); byte[] dataBeginBytes = new byte[Math.min(32, bufDup.limit() - bufDup.position())]; bufDup.get(dataBeginBytes); dataBegin = Bytes.toStringBinary(dataBeginBytes); @@ -521,7 +541,7 @@ public class HFileBlock implements Cacheable { HFileBlockDecodingContext ctx = blockType == BlockType.ENCODED_DATA ? reader.getBlockDecodingContext() : reader.getDefaultBlockDecodingContext(); - ByteBuffer dup = this.buf.duplicate(); + MultiByteBuffer dup = this.buf.duplicate(); dup.position(this.headerSize()); dup = dup.slice(); ctx.prepareDecoding(unpacked.getOnDiskSizeWithoutHeader(), @@ -534,16 +554,14 @@ public class HFileBlock implements Cacheable { // Below call to copyFromBufferToBuffer() will try positional read/write from/to buffers when // any of the buffer is DBB. So we change the limit on a dup buffer. No copying just create // new BB objects - ByteBuffer inDup = this.buf.duplicate(); + MultiByteBuffer inDup = this.buf.duplicate(); inDup.limit(inDup.limit() + headerSize()); - ByteBuffer outDup = unpacked.buf.duplicate(); + MultiByteBuffer outDup = unpacked.buf.duplicate(); outDup.limit(outDup.limit() + unpacked.headerSize()); - ByteBufferUtils.copyFromBufferToBuffer( - outDup, - inDup, - this.onDiskDataSizeWithHeader, + outDup.put( unpacked.headerSize() + unpacked.uncompressedSizeWithoutHeader - + unpacked.totalChecksumBytes(), unpacked.headerSize()); + + unpacked.totalChecksumBytes(), inDup, this.onDiskDataSizeWithHeader, + unpacked.headerSize()); } return unpacked; } @@ -571,11 +589,11 @@ public class HFileBlock implements Cacheable { // Copy header bytes into newBuf. // newBuf is HBB so no issue in calling array() - ByteBuffer dup = buf.duplicate(); - dup.position(0); + MultiByteBuffer dup = buf.duplicate(); + dup.rewind(); dup.get(newBuf.array(), newBuf.arrayOffset(), headerSize); - buf = newBuf; + buf = new MultiByteBuffer(newBuf); // set limit to exclude next block's header buf.limit(headerSize + uncompressedSizeWithoutHeader + cksumBytes); } @@ -627,9 +645,9 @@ public class HFileBlock implements Cacheable { * @return a byte stream reading the data + checksum of this block */ public DataInputStream getByteStream() { - ByteBuffer dup = this.buf.duplicate(); + MultiByteBuffer dup = this.buf.duplicate(); dup.position(this.headerSize()); - return new DataInputStream(new ByteBufferInputStream(dup)); + return new DataInputStream(new MultiByteBufferInputStream(dup)); } @Override @@ -1663,7 +1681,7 @@ public class HFileBlock implements Cacheable { // contains the header of next block, so no need to set next // block's header in it. b = new HFileBlock(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader), - this.fileContext.isUseHBaseChecksum()); + this.fileContext.isUseHBaseChecksum()); b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize; @@ -1731,7 +1749,7 @@ public class HFileBlock implements Cacheable { @Override public void serialize(ByteBuffer destination) { - ByteBufferUtils.copyFromBufferToBuffer(destination, this.buf, 0, getSerializedLength() + MultiByteBuffer.copyFromBufferToBuffer(destination, this.buf, 0, getSerializedLength() - EXTRA_SERIALIZATION_SPACE); serializeExtraInfo(destination); } @@ -1780,7 +1798,7 @@ public class HFileBlock implements Cacheable { if (castedComparison.uncompressedSizeWithoutHeader != this.uncompressedSizeWithoutHeader) { return false; } - if (ByteBufferUtils.compareTo(this.buf, 0, this.buf.limit(), castedComparison.buf, 0, + if (MultiByteBuffer.compareTo(this.buf, 0, this.buf.limit(), castedComparison.buf, 0, castedComparison.buf.limit()) != 0) { return false; } @@ -1864,12 +1882,24 @@ public class HFileBlock implements Cacheable { return this.fileContext; } + public void setCacheType(CacheType cacheType) { + this.cacheType = cacheType; + } + + public CacheType getCacheType() { + return this.cacheType; + } + + public static enum CacheType { + L1_CACHED, L2_CACHED, NOT_CACHED; + } + /** * Convert the contents of the block header into a human readable string. * This is mostly helpful for debugging. This assumes that the block * has minor version > 0. */ - static String toStringHeader(ByteBuffer buf) throws IOException { + static String toStringHeader(MultiByteBuffer buf) throws IOException { byte[] magicBuf = new byte[Math.min(buf.limit() - buf.position(), BlockType.MAGIC_LENGTH)]; buf.get(magicBuf); BlockType bt = BlockType.parse(magicBuf, 0, BlockType.MAGIC_LENGTH); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java index 77266df..4d1b497 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -32,20 +32,24 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue; +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; 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.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader; -import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter; +import org.apache.hadoop.hbase.util.MultiByteBuffer; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.util.StringUtils; @@ -233,78 +237,85 @@ public class HFileBlockIndex { int lookupLevel = 1; // How many levels deep we are in our lookup. int index = -1; - HFileBlock block; + HFileBlock block = null; + boolean dataBlock = false; while (true) { - - if (currentBlock != null && currentBlock.getOffset() == currentOffset) - { - // Avoid reading the same block again, even with caching turned off. - // This is crucial for compaction-type workload which might have - // caching turned off. This is like a one-block cache inside the - // scanner. - block = currentBlock; - } else { - // Call HFile's caching block reader API. We always cache index - // blocks, otherwise we might get terrible performance. - boolean shouldCache = cacheBlocks || (lookupLevel < searchTreeLevel); - BlockType expectedBlockType; - if (lookupLevel < searchTreeLevel - 1) { - expectedBlockType = BlockType.INTERMEDIATE_INDEX; - } else if (lookupLevel == searchTreeLevel - 1) { - expectedBlockType = BlockType.LEAF_INDEX; + try { + if (currentBlock != null && currentBlock != null + && currentBlock.getOffset() == currentOffset) { + // Avoid reading the same block again, even with caching turned off. + // This is crucial for compaction-type workload which might have + // caching turned off. This is like a one-block cache inside the + // scanner. + block = currentBlock; } else { - // this also accounts for ENCODED_DATA - expectedBlockType = BlockType.DATA; + // Call HFile's caching block reader API. We always cache index + // blocks, otherwise we might get terrible performance. + boolean shouldCache = cacheBlocks || (lookupLevel < searchTreeLevel); + BlockType expectedBlockType; + if (lookupLevel < searchTreeLevel - 1) { + expectedBlockType = BlockType.INTERMEDIATE_INDEX; + } else if (lookupLevel == searchTreeLevel - 1) { + expectedBlockType = BlockType.LEAF_INDEX; + } else { + // this also accounts for ENCODED_DATA + expectedBlockType = BlockType.DATA; + } + block = cachingBlockReader.readBlock(currentOffset, currentOnDiskSize, shouldCache, + pread, isCompaction, true, expectedBlockType, expectedDataBlockEncoding); } - block = cachingBlockReader.readBlock(currentOffset, - currentOnDiskSize, shouldCache, pread, isCompaction, true, - expectedBlockType, expectedDataBlockEncoding); - } - if (block == null) { - throw new IOException("Failed to read block at offset " + - currentOffset + ", onDiskSize=" + currentOnDiskSize); - } + if (block == null) { + throw new IOException("Failed to read block at offset " + currentOffset + + ", onDiskSize=" + currentOnDiskSize); + } - // Found a data block, break the loop and check our level in the tree. - if (block.getBlockType().isData()) { - break; - } + // Found a data block, break the loop and check our level in the tree. + if (block.getBlockType().isData()) { + dataBlock = true; + break; + } - // Not a data block. This must be a leaf-level or intermediate-level - // index block. We don't allow going deeper than searchTreeLevel. - if (++lookupLevel > searchTreeLevel) { - throw new IOException("Search Tree Level overflow: lookupLevel="+ - lookupLevel + ", searchTreeLevel=" + searchTreeLevel); - } + // Not a data block. This must be a leaf-level or intermediate-level + // index block. We don't allow going deeper than searchTreeLevel. + if (++lookupLevel > searchTreeLevel) { + throw new IOException("Search Tree Level overflow: lookupLevel=" + lookupLevel + + ", searchTreeLevel=" + searchTreeLevel); + } - // Locate the entry corresponding to the given key in the non-root - // (leaf or intermediate-level) index block. - ByteBuffer buffer = block.getBufferWithoutHeader(); - index = locateNonRootIndexEntry(buffer, key, comparator); - if (index == -1) { - // This has to be changed - // For now change this to key value - KeyValue kv = KeyValueUtil.ensureKeyValue(key); - throw new IOException("The key " - + Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength()) - + " is before the" + " first key of the non-root index block " - + block); - } + // Locate the entry corresponding to the given key in the non-root + // (leaf or intermediate-level) index block. + MultiByteBuffer buffer = block.getBufferWithoutHeader(); + index = locateNonRootIndexEntry(buffer, key, comparator); + if (index == -1) { + // This has to be changed + // For now change this to key value + KeyValue kv = KeyValueUtil.ensureKeyValue(key); + throw new IOException("The key " + + Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength()) + + " is before the" + " first key of the non-root index block " + block); + } - currentOffset = buffer.getLong(); - currentOnDiskSize = buffer.getInt(); + currentOffset = buffer.getLong(); + currentOnDiskSize = buffer.getInt(); // Only update next indexed key if there is a next indexed key in the current level byte[] tmpNextIndexedKey = getNonRootIndexedKey(buffer, index + 1); if (tmpNextIndexedKey != null) { nextIndexedKey = new KeyValue.KeyOnlyKeyValue(tmpNextIndexedKey); + } + } finally { + if (!dataBlock) { + cachingBlockReader.returnBlock(block); + } } } if (lookupLevel != searchTreeLevel) { - throw new IOException("Reached a data block at level " + lookupLevel + - " but the number of levels is " + searchTreeLevel); + assert dataBlock == true; + cachingBlockReader.returnBlock(block); + throw new IOException("Reached a data block at level " + lookupLevel + + " but the number of levels is " + searchTreeLevel); } // set the next indexed key for the current block. @@ -339,14 +350,15 @@ public class HFileBlockIndex { midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false, true, BlockType.LEAF_INDEX, null); - ByteBuffer b = midLeafBlock.getBufferWithoutHeader(); + MultiByteBuffer b = midLeafBlock.getBufferWithoutHeader(); int numDataBlocks = b.getInt(); - int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 1)); - int keyLen = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - + int keyRelOffset = b.getIntStrictlyForward(Bytes.SIZEOF_INT * (midKeyEntry + 1)); + int keyLen = b.getIntStrictlyForward(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - keyRelOffset; int keyOffset = Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset + SECONDARY_INDEX_ENTRY_OVERHEAD; - targetMidKey = ByteBufferUtils.toBytes(b, keyOffset, keyLen); + targetMidKey = b.toBytes(keyOffset, keyLen); + cachingBlockReader.returnBlock(midLeafBlock); } else { // The middle of the root-level index. targetMidKey = blockKeys[rootCount / 2]; @@ -416,6 +428,27 @@ public class HFileBlockIndex { return i - 1; } + public int rootBlockContainingKey(final ByteBuffer key, int offset, int length) { + int pos = Bytes.binarySearch(blockKeys, key, offset, length, comparator); + // pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see + // binarySearch's javadoc. + + if (pos >= 0) { + // This means this is an exact match with an element of blockKeys. + assert pos < blockKeys.length; + return pos; + } + + // Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i], + // and i is in [0, blockKeys.length]. We are returning j = i - 1 such that + // blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if + // key < blockKeys[0], meaning the file does not contain the given key. + + int i = -pos - 1; + assert 0 <= i && i <= blockKeys.length; + return i - 1; + } + /** * Finds the root-level index block containing the given key. * @@ -463,7 +496,7 @@ public class HFileBlockIndex { * @param i the ith position * @return The indexed key at the ith position in the nonRootIndex. */ - private byte[] getNonRootIndexedKey(ByteBuffer nonRootIndex, int i) { + private byte[] getNonRootIndexedKey(MultiByteBuffer nonRootIndex, int i) { int numEntries = nonRootIndex.getInt(0); if (i < 0 || i >= numEntries) { return null; @@ -487,7 +520,7 @@ public class HFileBlockIndex { int targetKeyLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (i + 2)) - targetKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD; - return ByteBufferUtils.toBytes(nonRootIndex, targetKeyOffset, targetKeyLength); + return nonRootIndex.toBytes(targetKeyOffset, targetKeyLength); } /** @@ -506,10 +539,9 @@ public class HFileBlockIndex { * -1 otherwise * @throws IOException */ - static int binarySearchNonRootIndex(Cell key, ByteBuffer nonRootIndex, + static int binarySearchNonRootIndex(Cell key, MultiByteBuffer nonRootIndex, KVComparator comparator) { - - int numEntries = nonRootIndex.getInt(0); + int numEntries = nonRootIndex.getIntStrictlyForward(0); int low = 0; int high = numEntries - 1; int mid = 0; @@ -521,13 +553,11 @@ public class HFileBlockIndex { // If we imagine that keys[-1] = -Infinity and // keys[numEntries] = Infinity, then we are maintaining an invariant that // keys[low - 1] < key < keys[high + 1] while narrowing down the range. - KeyValue.KeyOnlyKeyValue nonRootIndexKV = new KeyValue.KeyOnlyKeyValue(); while (low <= high) { mid = (low + high) >>> 1; // Midkey's offset relative to the end of secondary index - int midKeyRelOffset = nonRootIndex.getInt( - Bytes.SIZEOF_INT * (mid + 1)); + int midKeyRelOffset = nonRootIndex.getIntStrictlyForward(Bytes.SIZEOF_INT * (mid + 1)); // The offset of the middle key in the blockIndex buffer int midKeyOffset = entriesOffset // Skip secondary index @@ -537,16 +567,14 @@ public class HFileBlockIndex { // We subtract the two consecutive secondary index elements, which // gives us the size of the whole (offset, onDiskSize, key) tuple. We // then need to subtract the overhead of offset and onDiskSize. - int midLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (mid + 2)) - + int midLength = nonRootIndex.getIntStrictlyForward(Bytes.SIZEOF_INT * (mid + 2)) - midKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD; // we have to compare in this order, because the comparator order // has special logic when the 'left side' is a special key. - // TODO make KeyOnlyKeyValue to be Buffer backed and avoid array() call. This has to be - // done after HBASE-12224 & HBASE-12282 - nonRootIndexKV.setKey(nonRootIndex.array(), - nonRootIndex.arrayOffset() + midKeyOffset, midLength); - int cmp = comparator.compareOnlyKeyPortion(key, nonRootIndexKV); + Pair p = nonRootIndex.asSubBuffer(midKeyOffset, midLength); + Cell midCell = new ByteBufferBackedKeyOnlyKeyValue(p.getFirst(), p.getSecond(), midLength); + int cmp = comparator.compareOnlyKeyPortion(key, midCell); // key lives above the midpoint if (cmp > 0) @@ -595,20 +623,20 @@ public class HFileBlockIndex { * return -1 in the case the given key is before the first key. * */ - static int locateNonRootIndexEntry(ByteBuffer nonRootBlock, Cell key, + static int locateNonRootIndexEntry(MultiByteBuffer nonRootBlock, Cell key, KVComparator comparator) { int entryIndex = binarySearchNonRootIndex(key, nonRootBlock, comparator); if (entryIndex != -1) { - int numEntries = nonRootBlock.getInt(0); + int numEntries = nonRootBlock.getIntStrictlyForward(0); // The end of secondary index and the beginning of entries themselves. int entriesOffset = Bytes.SIZEOF_INT * (numEntries + 2); // The offset of the entry we are interested in relative to the end of // the secondary index. - int entryRelOffset = nonRootBlock.getInt(Bytes.SIZEOF_INT * (1 + entryIndex)); - + int entryRelOffset = nonRootBlock + .getIntStrictlyForward(Bytes.SIZEOF_INT * (1 + entryIndex)); nonRootBlock.position(entriesOffset + entryRelOffset); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index 7b92df9..91545e8 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -39,8 +39,6 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.cli.PosixParser; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -52,16 +50,22 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.BloomFilter; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.ByteBloomFilter; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Writables; @@ -334,14 +338,14 @@ public class HFilePrettyPrinter extends Configured implements Tool { System.out.print("K: " + cell); if (printValue) { System.out.print(" V: " - + Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(), + + ByteBufferUtils.toStringBinary(cell.getValueBuffer(), cell.getValueOffset(), cell.getValueLength())); int i = 0; - List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + List tags = TagUtil.asList(cell.getTagsBuffer(), cell.getTagsOffset(), cell.getTagsLength()); for (Tag tag : tags) { System.out.print(String.format(" T[%d]: %s", i++, - Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()))); + ByteBufferUtils.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()))); } } System.out.println(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java index c0e3e91..32f82c5 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; @@ -38,10 +39,13 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext; +import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IdLock; +import org.apache.hadoop.hbase.util.MultiByteBuffer; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.WritableUtils; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; @@ -72,6 +76,7 @@ public class HFileReaderV2 extends AbstractHFileReader { protected boolean includesMemstoreTS = false; protected boolean decodeMemstoreTS = false; + protected boolean shouldIncludeMemstoreTS() { return includesMemstoreTS; } @@ -109,12 +114,17 @@ public class HFileReaderV2 extends AbstractHFileReader { /** * Opens a HFile. You must load the index before you can use it by calling * {@link #loadFileInfo()}. - * - * @param path Path to HFile. - * @param trailer File trailer. - * @param fsdis input stream. - * @param size Length of the stream. - * @param cacheConf Cache configuration. + * + * @param path + * Path to HFile. + * @param trailer + * File trailer. + * @param fsdis + * input stream. + * @param size + * Length of the stream. + * @param cacheConf + * Cache configuration. * @param hfs * @param conf */ @@ -126,32 +136,28 @@ public class HFileReaderV2 extends AbstractHFileReader { trailer.expectMajorVersion(getMajorVersion()); validateMinorVersion(path, trailer.getMinorVersion()); this.hfileContext = createHFileContext(fsdis, fileSize, hfs, path, trailer); - HFileBlock.FSReaderImpl fsBlockReaderV2 = - new HFileBlock.FSReaderImpl(fsdis, fileSize, hfs, path, hfileContext); + HFileBlock.FSReaderImpl fsBlockReaderV2 = new HFileBlock.FSReaderImpl(fsdis, fileSize, hfs, + path, hfileContext); this.fsBlockReader = fsBlockReaderV2; // upcast // Comparator class name is stored in the trailer in version 2. comparator = trailer.createComparator(); dataBlockIndexReader = new HFileBlockIndex.BlockIndexReader(comparator, trailer.getNumDataIndexLevels(), this); - metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader( - KeyValue.RAW_COMPARATOR, 1); + metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader(KeyValue.RAW_COMPARATOR, 1); // Parse load-on-open data. HFileBlock.BlockIterator blockIter = fsBlockReaderV2.blockRange( - trailer.getLoadOnOpenDataOffset(), - fileSize - trailer.getTrailerSize()); + trailer.getLoadOnOpenDataOffset(), fileSize - trailer.getTrailerSize()); // Data index. We also read statistics about the block index written after // the root level. dataBlockIndexReader.readMultiLevelIndexRoot( - blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), - trailer.getDataIndexCount()); + blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), trailer.getDataIndexCount()); // Meta index. - metaBlockIndexReader.readRootIndex( - blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), + metaBlockIndexReader.readRootIndex(blockIter.nextBlockWithBlockType(BlockType.ROOT_INDEX), trailer.getMetaIndexCount()); // File info @@ -162,11 +168,9 @@ public class HFileReaderV2 extends AbstractHFileReader { lastKey = fileInfo.get(FileInfo.LASTKEY); avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN)); avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN)); - byte [] keyValueFormatVersion = - fileInfo.get(HFileWriterV2.KEY_VALUE_VERSION); - includesMemstoreTS = keyValueFormatVersion != null && - Bytes.toInt(keyValueFormatVersion) == - HFileWriterV2.KEY_VALUE_VER_WITH_MEMSTORE; + byte[] keyValueFormatVersion = fileInfo.get(HFileWriterV2.KEY_VALUE_VERSION); + includesMemstoreTS = keyValueFormatVersion != null + && Bytes.toInt(keyValueFormatVersion) == HFileWriterV2.KEY_VALUE_VER_WITH_MEMSTORE; fsBlockReaderV2.setIncludesMemstoreTS(includesMemstoreTS); if (includesMemstoreTS) { decodeMemstoreTS = Bytes.toLong(fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY)) > 0; @@ -198,8 +202,12 @@ public class HFileReaderV2 extends AbstractHFileReader { if (prevBlock != null) { onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader(); } + // Now here by default the count in the block cache would be + // incremented. Need + // to fetch ensure when we change the reference we decrement it. HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false, - null, null); + null, null); + returnBlock(block); prevBlock = block; offset += block.getOnDiskSizeWithHeader(); } @@ -221,11 +229,10 @@ public class HFileReaderV2 extends AbstractHFileReader { protected HFileContext createHFileContext(FSDataInputStreamWrapper fsdis, long fileSize, HFileSystem hfs, Path path, FixedFileTrailer trailer) throws IOException { - return new HFileContextBuilder() - .withIncludesMvcc(this.includesMemstoreTS) - .withCompression(this.compressAlgo) - .withHBaseCheckSum(trailer.getMinorVersion() >= MINOR_VERSION_WITH_CHECKSUM) - .build(); + return new HFileContextBuilder().withIncludesMvcc(this.includesMemstoreTS) + .withCompression(this.compressAlgo) + .withHBaseCheckSum(trailer.getMinorVersion() >= MINOR_VERSION_WITH_CHECKSUM) + .withPathName(path.getName()).build(); } /** @@ -233,92 +240,100 @@ public class HFileReaderV2 extends AbstractHFileReader { * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is * nothing to clean up in a Scanner. Letting go of your references to the * scanner is sufficient. - * - * @param cacheBlocks True if we should cache blocks read in by this scanner. - * @param pread Use positional read rather than seek+read if true (pread is - * better for random reads, seek+read is better scanning). - * @param isCompaction is scanner being used for a compaction? + * + * @param cacheBlocks + * True if we should cache blocks read in by this scanner. + * @param pread + * Use positional read rather than seek+read if true (pread is better + * for random reads, seek+read is better scanning). + * @param isCompaction + * is scanner being used for a compaction? * @return Scanner on this file. */ - @Override - public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, + @Override + public HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction) { if (dataBlockEncoder.useEncodedScanner()) { - return new EncodedScannerV2(this, cacheBlocks, pread, isCompaction, - hfileContext); + return new EncodedScannerV2(this, cacheBlocks, pread, isCompaction, hfileContext); } - return new ScannerV2(this, cacheBlocks, pread, isCompaction); + ScannerV2 scannerV2 = new ScannerV2(this, cacheBlocks, pread, isCompaction); + return scannerV2; } /** - * Retrieve block from cache. Validates the retrieved block's type vs {@code expectedBlockType} - * and its encoding vs. {@code expectedDataBlockEncoding}. Unpacks the block as necessary. + * Retrieve block from cache. Validates the retrieved block's type vs + * {@code expectedBlockType} and its encoding vs. + * {@code expectedDataBlockEncoding}. Unpacks the block as necessary. */ - private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock, - boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, - DataBlockEncoding expectedDataBlockEncoding) throws IOException { - // Check cache for block. If found return. - if (cacheConf.isBlockCacheEnabled()) { - BlockCache cache = cacheConf.getBlockCache(); - HFileBlock cachedBlock = (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock, - updateCacheMetrics); - if (cachedBlock != null) { - if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) { - cachedBlock = cachedBlock.unpack(hfileContext, fsBlockReader); - } - validateBlockType(cachedBlock, expectedBlockType); - - if (expectedDataBlockEncoding == null) { - return cachedBlock; - } - DataBlockEncoding actualDataBlockEncoding = - cachedBlock.getDataBlockEncoding(); - // Block types other than data blocks always have - // DataBlockEncoding.NONE. To avoid false negative cache misses, only - // perform this check if cached block is a data block. - if (cachedBlock.getBlockType().isData() && - !actualDataBlockEncoding.equals(expectedDataBlockEncoding)) { - // This mismatch may happen if a ScannerV2, which is used for say a - // compaction, tries to read an encoded block from the block cache. - // The reverse might happen when an EncodedScannerV2 tries to read - // un-encoded blocks which were cached earlier. - // - // Because returning a data block with an implicit BlockType mismatch - // will cause the requesting scanner to throw a disk read should be - // forced here. This will potentially cause a significant number of - // cache misses, so update so we should keep track of this as it might - // justify the work on a CompoundScannerV2. - if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) && - !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) { - // If the block is encoded but the encoding does not match the - // expected encoding it is likely the encoding was changed but the - // block was not yet evicted. Evictions on file close happen async - // so blocks with the old encoding still linger in cache for some - // period of time. This event should be rare as it only happens on - // schema definition change. - LOG.info("Evicting cached block with key " + cacheKey + - " because of a data block encoding mismatch" + - "; expected: " + expectedDataBlockEncoding + - ", actual: " + actualDataBlockEncoding); - cache.evictBlock(cacheKey); - } - return null; - } - return cachedBlock; - } - } - return null; - } + private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock, + boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, + DataBlockEncoding expectedDataBlockEncoding) throws IOException { + // Check cache for block. If found return. + if (cacheConf.isBlockCacheEnabled()) { + BlockCache cache = cacheConf.getBlockCache(); + HFileBlock cachedBlock = (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock, + updateCacheMetrics); + if (cachedBlock != null) { + if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) { + HFileBlock compressedBlock = cachedBlock; + cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader); + if (compressedBlock != cachedBlock) cache.returnBlock(cacheKey, compressedBlock); + } + validateBlockType(cachedBlock, expectedBlockType); + + if (expectedDataBlockEncoding == null) { + return cachedBlock; + } + DataBlockEncoding actualDataBlockEncoding = cachedBlock.getDataBlockEncoding(); + // Block types other than data blocks always have + // DataBlockEncoding.NONE. To avoid false negative cache misses, only + // perform this check if cached block is a data block. + if (cachedBlock.getBlockType().isData() + && !actualDataBlockEncoding.equals(expectedDataBlockEncoding)) { + // This mismatch may happen if a ScannerV2, which is used for say a + // compaction, tries to read an encoded block from the block cache. + // The reverse might happen when an EncodedScannerV2 tries to read + // un-encoded blocks which were cached earlier. + // + // Because returning a data block with an implicit BlockType mismatch + // will cause the requesting scanner to throw a disk read should be + // forced here. This will potentially cause a significant number of + // cache misses, so update so we should keep track of this as it might + // justify the work on a CompoundScannerV2. + if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) + && !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) { + // If the block is encoded but the encoding does not match the + // expected encoding it is likely the encoding was changed but the + // block was not yet evicted. Evictions on file close happen async + // so blocks with the old encoding still linger in cache for some + // period of time. This event should be rare as it only happens on + // schema definition change. + LOG.info("Evicting cached block with key " + cacheKey + + " because of a data block encoding mismatch" + "; expected: " + + expectedDataBlockEncoding + ", actual: " + actualDataBlockEncoding); + // This is an error scenario. so here we need to decrement the + // count. + cache.returnBlock(cacheKey, cachedBlock); + cache.evictBlock(cacheKey); + } + return null; + } + return cachedBlock; + } + } + return null; + } + /** * @param metaBlockName - * @param cacheBlock Add block to cache, if found + * @param cacheBlock + * Add block to cache, if found * @return block wrapped in a ByteBuffer, with header skipped * @throws IOException */ @Override - public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock) - throws IOException { + public HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException { if (trailer.getMetaIndexCount() == 0) { return null; // there are no meta blocks } @@ -327,8 +342,7 @@ public class HFileReaderV2 extends AbstractHFileReader { } byte[] mbname = Bytes.toBytes(metaBlockName); - int block = metaBlockIndexReader.rootBlockContainingKey(mbname, - 0, mbname.length); + int block = metaBlockIndexReader.rootBlockContainingKey(mbname, 0, mbname.length); if (block == -1) return null; long blockSize = metaBlockIndexReader.getRootBlockDataSize(block); @@ -344,26 +358,26 @@ public class HFileReaderV2 extends AbstractHFileReader { cacheBlock &= cacheConf.shouldCacheDataOnRead(); if (cacheConf.isBlockCacheEnabled()) { HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, false, true, true, - BlockType.META, null); + BlockType.META, null); if (cachedBlock != null) { assert cachedBlock.isUnpacked() : "Packed block leak."; // Return a distinct 'shallow copy' of the block, // so pos does not get messed by the scanner - return cachedBlock.getBufferWithoutHeader(); + return cachedBlock; } // Cache Miss, please load. } - HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, - blockSize, -1, true).unpack(hfileContext, fsBlockReader); + HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, blockSize, -1, true) + .unpack(hfileContext, fsBlockReader); // Cache the block if (cacheBlock) { - cacheConf.getBlockCache().cacheBlock(cacheKey, metaBlock, - cacheConf.isInMemory(), this.cacheConf.isCacheDataInL1()); + cacheConf.getBlockCache().cacheBlock(cacheKey, metaBlock, cacheConf.isInMemory(), + this.cacheConf.isCacheDataInL1()); } - return metaBlock.getBufferWithoutHeader(); + return metaBlock; } } @@ -371,17 +385,17 @@ public class HFileReaderV2 extends AbstractHFileReader { public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, final boolean cacheBlock, boolean pread, final boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, - DataBlockEncoding expectedDataBlockEncoding) - throws IOException { + DataBlockEncoding expectedDataBlockEncoding) throws IOException { if (dataBlockIndexReader == null) { throw new IOException("Block index not loaded"); } if (dataBlockOffset < 0 || dataBlockOffset >= trailer.getLoadOnOpenDataOffset()) { - throw new IOException("Requested block is out of range: " + dataBlockOffset + - ", lastDataBlockOffset: " + trailer.getLastDataBlockOffset()); + throw new IOException("Requested block is out of range: " + dataBlockOffset + + ", lastDataBlockOffset: " + trailer.getLastDataBlockOffset()); } - // For any given block from any given file, synchronize reads for said block. + // For any given block from any given file, synchronize reads for said + // block. // Without a cache, this synchronizing is needless overhead, but really // the other choice is to duplicate work (which the cache would prevent you // from doing). @@ -397,10 +411,12 @@ public class HFileReaderV2 extends AbstractHFileReader { // Check cache for block. If found return. if (cacheConf.isBlockCacheEnabled()) { - // Try and get the block from the block cache. If the useLock variable is true then this - // is the second time through the loop and it should not be counted as a block cache miss. + // Try and get the block from the block cache. If the useLock variable + // is true then this + // is the second time through the loop and it should not be counted as + // a block cache miss. HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, useLock, isCompaction, - updateCacheMetrics, expectedBlockType, expectedDataBlockEncoding); + updateCacheMetrics, expectedBlockType, expectedDataBlockEncoding); if (cachedBlock != null) { if (Trace.isTracing()) { traceScope.getSpan().addTimelineAnnotation("blockCacheHit"); @@ -411,11 +427,12 @@ public class HFileReaderV2 extends AbstractHFileReader { HFile.dataBlockReadCnt.incrementAndGet(); } // Validate encoding type for data blocks. We include encoding - // type in the cache key, and we expect it to match on a cache hit. + // type in the cache key, and we expect it to match on a cache + // hit. if (cachedBlock.getDataBlockEncoding() != dataBlockEncoder.getDataBlockEncoding()) { throw new IOException("Cached block under key " + cacheKey + " " - + "has wrong encoding: " + cachedBlock.getDataBlockEncoding() + " (expected: " - + dataBlockEncoder.getDataBlockEncoding() + ")"); + + "has wrong encoding: " + cachedBlock.getDataBlockEncoding() + " (expected: " + + dataBlockEncoder.getDataBlockEncoding() + ")"); } } // Cache-hit. Return! @@ -441,8 +458,8 @@ public class HFileReaderV2 extends AbstractHFileReader { // Cache the block if necessary if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) { cacheConf.getBlockCache().cacheBlock(cacheKey, - cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked, - cacheConf.isInMemory(), this.cacheConf.isCacheDataInL1()); + cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked, + cacheConf.isInMemory(), this.cacheConf.isCacheDataInL1()); } if (updateCacheMetrics && hfileBlock.getBlockType().isData()) { @@ -466,15 +483,16 @@ public class HFileReaderV2 extends AbstractHFileReader { /** * Compares the actual type of a block retrieved from cache or disk with its - * expected type and throws an exception in case of a mismatch. Expected - * block type of {@link BlockType#DATA} is considered to match the actual - * block type [@link {@link BlockType#ENCODED_DATA} as well. - * @param block a block retrieved from cache or disk - * @param expectedBlockType the expected block type, or null to skip the - * check + * expected type and throws an exception in case of a mismatch. Expected block + * type of {@link BlockType#DATA} is considered to match the actual block type + * [@link {@link BlockType#ENCODED_DATA} as well. + * + * @param block + * a block retrieved from cache or disk + * @param expectedBlockType + * the expected block type, or null to skip the check */ - private void validateBlockType(HFileBlock block, - BlockType expectedBlockType) throws IOException { + private void validateBlockType(HFileBlock block, BlockType expectedBlockType) throws IOException { if (expectedBlockType == null) { return; } @@ -485,8 +503,8 @@ public class HFileReaderV2 extends AbstractHFileReader { return; } if (actualBlockType != expectedBlockType) { - throw new IOException("Expected block type " + expectedBlockType + ", " + - "but got " + actualBlockType + ": " + block); + throw new IOException("Expected block type " + expectedBlockType + ", " + "but got " + + actualBlockType + ": " + block); } } @@ -520,8 +538,7 @@ public class HFileReaderV2 extends AbstractHFileReader { if (evictOnClose && cacheConf.isBlockCacheEnabled()) { int numEvicted = cacheConf.getBlockCache().evictBlocksByHfileName(name); if (LOG.isTraceEnabled()) { - LOG.trace("On close, file=" + name + " evicted=" + numEvicted - + " block(s)"); + LOG.trace("On close, file=" + name + " evicted=" + numEvicted + " block(s)"); } } fsBlockReader.closeStreams(); @@ -537,27 +554,66 @@ public class HFileReaderV2 extends AbstractHFileReader { return fsBlockReader; } + /** + * Handles block reference under usage and ensures the prev Block under use is + * returned back in case of Bucket Cache + */ + + protected static class CurrentBlockHolder { + private HFileBlock curBlock; + private CachingBlockReader reader; + + public CurrentBlockHolder(CachingBlockReader reader) { + this.reader = reader; + } + + // Better name + public void updateCurrentBlock(HFileBlock block) { + if (block != null && this.curBlock != null) { + if (block.getOffset() != this.curBlock.getOffset()) { + returnBlockToCache(this.curBlock); + } + } + this.curBlock = block; + } + + public HFileBlock getCurrentBlock() { + return this.curBlock; + } - protected abstract static class AbstractScannerV2 - extends AbstractHFileReader.Scanner { - protected HFileBlock block; + public void reset() { + returnBlockToCache(curBlock); + this.curBlock = null; + } + private void returnBlockToCache(HFileBlock block) { + this.reader.returnBlock(block); + } + } + protected abstract static class AbstractScannerV2 extends AbstractHFileReader.Scanner { + + //protected HFileBlock block; + // Holds the current active block. + protected CurrentBlockHolder blockHolder; + @Override public Cell getNextIndexedKey() { return nextIndexedKey; } /** - * The next indexed key is to keep track of the indexed key of the next data block. - * If the nextIndexedKey is HConstants.NO_NEXT_INDEXED_KEY, it means that the - * current data block is the last data block. - * - * If the nextIndexedKey is null, it means the nextIndexedKey has not been loaded yet. + * The next indexed key is to keep track of the indexed key of the next data + * block. If the nextIndexedKey is HConstants.NO_NEXT_INDEXED_KEY, it means + * that the current data block is the last data block. + * + * If the nextIndexedKey is null, it means the nextIndexedKey has not been + * loaded yet. */ protected Cell nextIndexedKey; - public AbstractScannerV2(HFileReaderV2 r, boolean cacheBlocks, - final boolean pread, final boolean isCompaction) { + public AbstractScannerV2(HFileReaderV2 r, boolean cacheBlocks, final boolean pread, + final boolean isCompaction) { super(r, cacheBlocks, pread, isCompaction); + blockHolder = new CurrentBlockHolder(reader); } protected abstract ByteBuffer getFirstKeyInBlock(HFileBlock curBlock); @@ -592,51 +648,115 @@ public class HFileReaderV2 extends AbstractHFileReader { // don't do anything. return compared; } else { - // The comparison with no_next_index_key has to be checked - if (this.nextIndexedKey != null && - (this.nextIndexedKey == HConstants.NO_NEXT_INDEXED_KEY || reader - .getComparator() - .compareOnlyKeyPortion(key, nextIndexedKey) < 0)) { - // The reader shall continue to scan the current data block instead - // of querying the - // block index as long as it knows the target key is strictly - // smaller than - // the next indexed key or the current data block is the last data - // block. - return loadBlockAndSeekToKey(this.block, nextIndexedKey, false, key, false); + try { + // The comparison with no_next_index_key has to be checked + if (this.nextIndexedKey != null && + (this.nextIndexedKey == HConstants.NO_NEXT_INDEXED_KEY || reader + .getComparator() + .compareOnlyKeyPortion(key, nextIndexedKey) < 0)) { + // The reader shall continue to scan the current data block + // instead + // of querying the + // block index as long as it knows the target key is strictly + // smaller than + // the next indexed key or the current data block is the last data + // block. + return loadBlockAndSeekToKey(blockHolder.getCurrentBlock(), nextIndexedKey, false, + key, false); + } + } catch (Exception e) { + this.reader.returnBlock(this.blockHolder.getCurrentBlock()); + throw e; } } } // Don't rewind on a reseek operation, because reseek implies that we are // always going forward in the file. + // SeekTo will decrement the return the block on error - so need not cover + // the try/catch here return seekTo(key, false); } + protected abstract void setNonSeekedState(); + + protected abstract void updateCurrentBlock(HFileBlock block) throws IOException; + + /** + * Positions this scanner at the start of the file. + * + * @return false if empty file; i.e. a call to next would return false and + * the current key and value are undefined. + * @throws IOException + */ + @Override + public boolean seekTo() throws IOException { + try { + if (reader == null) { + return false; + } + + if (reader.getTrailer().getEntryCount() == 0) { + // No data blocks. + return false; + } + long firstDataBlockOffset = reader.getTrailer().getFirstDataBlockOffset(); + if (blockHolder.getCurrentBlock() != null + && blockHolder.getCurrentBlock().getOffset() == firstDataBlockOffset) { + return processFirstDataBlock(); + } + readAndUpdateNewBlock(firstDataBlockOffset); + return true; + } catch (Exception e) { + this.reader.returnBlock(this.blockHolder.getCurrentBlock()); + throw e; + } + } + + protected abstract boolean processFirstDataBlock() throws IOException; + + protected void readAndUpdateNewBlock(long firstDataBlockOffset) throws IOException, + CorruptHFileException { + HFileBlock newBlock = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread, + isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding()); + if (newBlock.getOffset() < 0) { + throw new IOException("Invalid block offset: " + newBlock.getOffset()); + } + updateCurrentBlock(newBlock); + } /** * An internal API function. Seek to the given key, optionally rewinding to * the first key of the block before doing the seek. - * - * @param key - a cell representing the key that we need to fetch - * @param rewind whether to rewind to the first key of the block before - * doing the seek. If this is false, we are assuming we never go - * back, otherwise the result is undefined. - * @return -1 if the key is earlier than the first key of the file, - * 0 if we are at the given key, 1 if we are past the given key - * -2 if the key is earlier than the first key of the file while - * using a faked index key + * + * @param key + * - a cell representing the key that we need to fetch + * @param rewind + * whether to rewind to the first key of the block before doing the + * seek. If this is false, we are assuming we never go back, + * otherwise the result is undefined. + * @return -1 if the key is earlier than the first key of the file, 0 if we + * are at the given key, 1 if we are past the given key -2 if the + * key is earlier than the first key of the file while using a faked + * index key * @throws IOException */ public int seekTo(Cell key, boolean rewind) throws IOException { - HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader(); - BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(key, block, - cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding()); - if (blockWithScanInfo == null || blockWithScanInfo.getHFileBlock() == null) { - // This happens if the key e.g. falls before the beginning of the file. - return -1; + try { + HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader(); + BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(key, + blockHolder.getCurrentBlock(), cacheBlocks, pread, isCompaction, + getEffectiveDataBlockEncoding()); + if (blockWithScanInfo == null || blockWithScanInfo == null) { + // This happens if the key e.g. falls before the beginning of the + // file. + return -1; + } + return loadBlockAndSeekToKey(blockWithScanInfo.getHFileBlock(), + blockWithScanInfo.getNextIndexedKey(), rewind, key, false); + } catch (Exception e) { + this.reader.returnBlock(this.blockHolder.getCurrentBlock()); + throw e; } - return loadBlockAndSeekToKey(blockWithScanInfo.getHFileBlock(), - blockWithScanInfo.getNextIndexedKey(), rewind, key, false); } @Override @@ -646,87 +766,109 @@ public class HFileReaderV2 extends AbstractHFileReader { @Override public boolean seekBefore(Cell key) throws IOException { - HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, block, - cacheBlocks, pread, isCompaction, - ((HFileReaderV2) reader).getEffectiveEncodingInCache(isCompaction)); - if (seekToBlock == null) { - return false; - } - ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock); - - if (reader.getComparator() - .compareOnlyKeyPortion( - new KeyValue.KeyOnlyKeyValue(firstKey.array(), firstKey.arrayOffset(), - firstKey.limit()), key) >= 0) { - long previousBlockOffset = seekToBlock.getPrevBlockOffset(); - // The key we are interested in - if (previousBlockOffset == -1) { - // we have a 'problem', the key we want is the first of the file. + try { + HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, + blockHolder.getCurrentBlock(), cacheBlocks, pread, isCompaction, + ((HFileReaderV2) reader).getEffectiveEncodingInCache(isCompaction)); + if (seekToBlock == null) { return false; } + ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock); + int res = 0; + if (firstKey.hasArray()) { + res = reader.getComparator().compareOnlyKeyPortion( + new KeyValue.KeyOnlyKeyValue(firstKey.array(), firstKey.arrayOffset(), + firstKey.limit()), key); + } else { + res = reader.getComparator().compareOnlyKeyPortion( + new ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue(firstKey, + firstKey.position(), firstKey.limit()), key); + } + if (res >= 0) { + long previousBlockOffset = seekToBlock.getPrevBlockOffset(); + // The key we are interested in + if (previousBlockOffset == -1) { + // we have a 'problem', the key we want is the first of the file. + return false; + } - // It is important that we compute and pass onDiskSize to the block - // reader so that it does not have to read the header separately to - // figure out the size. - seekToBlock = reader.readBlock(previousBlockOffset, - seekToBlock.getOffset() - previousBlockOffset, cacheBlocks, - pread, isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding()); - // TODO shortcut: seek forward in this block to the last key of the - // block. + // It is important that we compute and pass onDiskSize to the block + // reader so that it does not have to read the header separately to + // figure out the size. + reader.returnBlock(seekToBlock); + seekToBlock = reader.readBlock(previousBlockOffset, seekToBlock.getOffset() + - previousBlockOffset, cacheBlocks, pread, isCompaction, true, BlockType.DATA, + getEffectiveDataBlockEncoding()); + // TODO shortcut: seek forward in this block to the last key of the + // block. + } + Cell firstKeyInCurrentBlock = new KeyValue.KeyOnlyKeyValue(Bytes.getBytes(firstKey)); + loadBlockAndSeekToKey(seekToBlock, firstKeyInCurrentBlock, true, key, true); + return true; + } catch (Exception e) { + this.reader.returnBlock(this.blockHolder.getCurrentBlock()); + throw e; } - Cell firstKeyInCurrentBlock = new KeyValue.KeyOnlyKeyValue(Bytes.getBytes(firstKey)); - loadBlockAndSeekToKey(seekToBlock, firstKeyInCurrentBlock, true, key, true); - return true; } /** * Scans blocks in the "scanned" section of the {@link HFile} until the next * data block is found. - * + * * @return the next block, or null if there are no more data blocks * @throws IOException */ protected HFileBlock readNextDataBlock() throws IOException { long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); - if (block == null) + if (blockHolder.getCurrentBlock() == null) return null; - HFileBlock curBlock = block; + HFileBlock curBlock = blockHolder.getCurrentBlock(); do { - if (curBlock.getOffset() >= lastDataBlockOffset) + if (curBlock.getOffset() >= lastDataBlockOffset) { return null; + } if (curBlock.getOffset() < 0) { - throw new IOException("Invalid block file offset: " + block); + throw new IOException("Invalid block file offset: " + blockHolder.getCurrentBlock()); } // We are reading the next block without block type validation, because // it might turn out to be a non-data block. curBlock = reader.readBlock(curBlock.getOffset() - + curBlock.getOnDiskSizeWithHeader(), - curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread, - isCompaction, true, null, getEffectiveDataBlockEncoding()); + + curBlock.getOnDiskSizeWithHeader(), curBlock + .getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread, isCompaction, true, null, + getEffectiveDataBlockEncoding()); + if (curBlock != null && !curBlock.getBlockType().isData()) { + reader.returnBlock(curBlock); + } } while (!curBlock.getBlockType().isData()); - return curBlock; } public DataBlockEncoding getEffectiveDataBlockEncoding() { - return ((HFileReaderV2)reader).getEffectiveEncodingInCache(isCompaction); + return ((HFileReaderV2) reader).getEffectiveEncodingInCache(isCompaction); } + /** * Compare the given key against the current key + * * @param comparator * @param key * @param offset * @param length - * @return -1 is the passed key is smaller than the current key, 0 if equal and 1 if greater + * @return -1 is the passed key is smaller than the current key, 0 if equal + * and 1 if greater */ - public abstract int compareKey(KVComparator comparator, byte[] key, int offset, - int length); - public abstract int compareKey(KVComparator comparator, Cell kv); + + @Override + public void close() { + if (this.blockHolder.getCurrentBlock() != null) { + this.blockHolder.reset(); + } + } } /** @@ -735,8 +877,8 @@ public class HFileReaderV2 extends AbstractHFileReader { protected static class ScannerV2 extends AbstractScannerV2 { private HFileReaderV2 reader; - public ScannerV2(HFileReaderV2 r, boolean cacheBlocks, - final boolean pread, final boolean isCompaction) { + public ScannerV2(HFileReaderV2 r, boolean cacheBlocks, final boolean pread, + final boolean isCompaction) { super(r, cacheBlocks, pread, isCompaction); this.reader = r; } @@ -746,8 +888,9 @@ public class HFileReaderV2 extends AbstractHFileReader { if (!isSeeked()) return null; - KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset() - + blockBuffer.position(), getCellBufSize()); + int cellBufSize = getCellBufSize(); + ByteBuffer buf = blockBuffer.asSubBuffer(cellBufSize); + ByteBufferBackedKeyValue ret = new ByteBufferBackedKeyValue(buf, buf.position(), cellBufSize); if (this.reader.shouldIncludeMemstoreTS()) { ret.setSequenceId(currMemstoreTS); } @@ -761,29 +904,28 @@ public class HFileReaderV2 extends AbstractHFileReader { @Override public ByteBuffer getKey() { assertSeeked(); - return ByteBuffer.wrap( - blockBuffer.array(), - blockBuffer.arrayOffset() + blockBuffer.position() - + KEY_VALUE_LEN_SIZE, currKeyLen).slice(); - } - - @Override - public int compareKey(KVComparator comparator, byte[] key, int offset, int length) { - return comparator.compareFlatKey(key, offset, length, blockBuffer.array(), - blockBuffer.arrayOffset() + blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen); + ByteBuffer keyBuf = blockBuffer.asSubBuffer(KEY_VALUE_LEN_SIZE + currKeyLen); + int newPos = keyBuf.position() + KEY_VALUE_LEN_SIZE; + keyBuf = keyBuf.duplicate(); + keyBuf.position(newPos); + keyBuf.limit(newPos + currKeyLen); + return keyBuf.slice(); } @Override public ByteBuffer getValue() { assertSeeked(); - return ByteBuffer.wrap( - blockBuffer.array(), - blockBuffer.arrayOffset() + blockBuffer.position() - + KEY_VALUE_LEN_SIZE + currKeyLen, currValueLen).slice(); + Pair p = this.blockBuffer.asSubBuffer(blockBuffer.position() + + KEY_VALUE_LEN_SIZE + currKeyLen, currValueLen); + ByteBuffer valBuf = p.getFirst().duplicate(); + valBuf.position(p.getSecond()); + valBuf.limit(p.getSecond() + currValueLen); + return valBuf.slice(); } + @Override protected void setNonSeekedState() { - block = null; + blockHolder.reset(); blockBuffer = null; currKeyLen = 0; currValueLen = 0; @@ -795,7 +937,7 @@ public class HFileReaderV2 extends AbstractHFileReader { * Go to the next key/value in the block section. Loads the next block if * necessary. If successful, {@link #getKey()} and {@link #getValue()} can * be called. - * + * * @return true if successfully navigated to the next key/value */ @Override @@ -803,86 +945,62 @@ public class HFileReaderV2 extends AbstractHFileReader { assertSeeked(); try { - blockBuffer.position(getNextCellStartPosition()); - } catch (IllegalArgumentException e) { - LOG.error("Current pos = " + blockBuffer.position() - + "; currKeyLen = " + currKeyLen + "; currValLen = " - + currValueLen + "; block limit = " + blockBuffer.limit() - + "; HFile name = " + reader.getName() - + "; currBlock currBlockOffset = " + block.getOffset()); - throw e; - } + try { + blockBuffer.skip(getCurCellSize()); + } catch (IllegalArgumentException e) { + LOG.error("Current pos = " + blockBuffer.position() + "; currKeyLen = " + currKeyLen + + "; currValLen = " + currValueLen + "; block limit = " + blockBuffer.limit() + + "; HFile name = " + reader.getName() + "; currBlock currBlockOffset = " + + blockHolder.getCurrentBlock().getOffset()); + throw e; + } - if (blockBuffer.remaining() <= 0) { - long lastDataBlockOffset = - reader.getTrailer().getLastDataBlockOffset(); + // TODO : Avoid this explicit call - Try getting the remaing call when + // we do the position() in line number 845 + if (blockBuffer.hasRemaining() == false) { + long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); - if (block.getOffset() >= lastDataBlockOffset) { - setNonSeekedState(); - return false; - } + if (blockHolder.getCurrentBlock().getOffset() >= lastDataBlockOffset) { + setNonSeekedState(); + return false; + } - // read the next block - HFileBlock nextBlock = readNextDataBlock(); - if (nextBlock == null) { - setNonSeekedState(); - return false; + // read the next block + HFileBlock nextBlock = readNextDataBlock(); + if (nextBlock == null) { + setNonSeekedState(); + return false; + } + + updateCurrentBlock(nextBlock); + return true; } - updateCurrBlock(nextBlock); + // We are still in the same block. + readKeyValueLen(); return true; + } catch (Exception e) { + this.reader.returnBlock(this.blockHolder.getCurrentBlock()); + throw e; } - - // We are still in the same block. - readKeyValueLen(); - return true; } - protected int getNextCellStartPosition() { - return blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen - + currMemstoreTSLen; + protected int getCurCellSize() { + return KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen + currMemstoreTSLen; } - /** - * Positions this scanner at the start of the file. - * - * @return false if empty file; i.e. a call to next would return false and - * the current key and value are undefined. - * @throws IOException - */ - @Override - public boolean seekTo() throws IOException { - if (reader == null) { - return false; - } - - if (reader.getTrailer().getEntryCount() == 0) { - // No data blocks. - return false; - } - - long firstDataBlockOffset = - reader.getTrailer().getFirstDataBlockOffset(); - if (block != null && block.getOffset() == firstDataBlockOffset) { - blockBuffer.rewind(); - readKeyValueLen(); - return true; - } - - block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread, - isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding()); - if (block.getOffset() < 0) { - throw new IOException("Invalid block offset: " + block.getOffset()); - } - updateCurrBlock(block); + protected boolean processFirstDataBlock() { + blockBuffer.rewind(); + readKeyValueLen(); return true; } @Override protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey, boolean rewind, Cell key, boolean seekBefore) throws IOException { - if (block == null || block.getOffset() != seekToBlock.getOffset()) { - updateCurrBlock(seekToBlock); + if (blockHolder.getCurrentBlock() == null + || blockHolder.getCurrentBlock().getOffset() != seekToBlock.getOffset()) { + updateCurrentBlock(seekToBlock); } else if (rewind) { blockBuffer.rewind(); } @@ -895,22 +1013,22 @@ public class HFileReaderV2 extends AbstractHFileReader { /** * Updates the current block to be the given {@link HFileBlock}. Seeks to * the the first key/value pair. - * - * @param newBlock the block to make current + * + * @param newBlock + * the block to make current */ - protected void updateCurrBlock(HFileBlock newBlock) { - block = newBlock; - + @Override + protected void updateCurrentBlock(HFileBlock newBlock) { + // Set the active block on the reader // sanity check - if (block.getBlockType() != BlockType.DATA) { - throw new IllegalStateException("ScannerV2 works only on data " + - "blocks, got " + block.getBlockType() + "; " + - "fileName=" + reader.name + ", " + - "dataBlockEncoder=" + reader.dataBlockEncoder + ", " + - "isCompaction=" + isCompaction); + if (newBlock.getBlockType() != BlockType.DATA) { + throw new IllegalStateException("ScannerV2 works only on data " + "blocks, got " + + blockHolder.getCurrentBlock().getBlockType() + "; " + "fileName=" + reader.name + ", " + + "dataBlockEncoder=" + reader.dataBlockEncoder + ", " + "isCompaction=" + isCompaction); } - blockBuffer = block.getBufferWithoutHeader(); + blockHolder.updateCurrentBlock(newBlock); + blockBuffer = newBlock.getBufferWithoutHeader(); readKeyValueLen(); blockFetches++; @@ -922,15 +1040,14 @@ public class HFileReaderV2 extends AbstractHFileReader { blockBuffer.mark(); currKeyLen = blockBuffer.getInt(); currValueLen = blockBuffer.getInt(); - ByteBufferUtils.skip(blockBuffer, currKeyLen + currValueLen); + blockBuffer.skip(currKeyLen + currValueLen); readMvccVersion(); - if (currKeyLen < 0 || currValueLen < 0 - || currKeyLen > blockBuffer.limit() + if (currKeyLen < 0 || currValueLen < 0 || currKeyLen > blockBuffer.limit() || currValueLen > blockBuffer.limit()) { - throw new IllegalStateException("Invalid currKeyLen " + currKeyLen - + " or currValueLen " + currValueLen + ". Block offset: " - + block.getOffset() + ", block length: " + blockBuffer.limit() - + ", position: " + blockBuffer.position() + " (without header)."); + throw new IllegalStateException("Invalid currKeyLen " + currKeyLen + " or currValueLen " + + currValueLen + ". Block offset: " + blockHolder.getCurrentBlock().getOffset() + + ", block length: " + blockBuffer.limit() + ", position: " + blockBuffer.position() + + " (without header)."); } blockBuffer.reset(); } @@ -939,8 +1056,12 @@ public class HFileReaderV2 extends AbstractHFileReader { if (this.reader.shouldIncludeMemstoreTS()) { if (this.reader.decodeMemstoreTS) { try { - currMemstoreTS = Bytes.readVLong(blockBuffer.array(), blockBuffer.arrayOffset() - + blockBuffer.position()); + // TODO we assume hasArray + /* + * currMemstoreTS = Bytes.readVLong(blockBuffer.array(), + * blockBuffer.arrayOffset() + blockBuffer.position()); + */ + currMemstoreTS = blockBuffer.getVLong(); currMemstoreTSLen = WritableUtils.getVIntSize(currMemstoreTS); } catch (Exception e) { throw new RuntimeException("Error reading memstore timestamp", e); @@ -955,11 +1076,11 @@ public class HFileReaderV2 extends AbstractHFileReader { /** * Within a loaded block, seek looking for the last key that is smaller than * (or equal to?) the key we are interested in. - * + * * A note on the seekBefore: if you have seekBefore = true, AND the first * key in the block = key, then you'll get thrown exceptions. The caller has * to check for that case and load the previous block as appropriate. - * + * * @param key * the key to find * @param seekBefore @@ -974,18 +1095,20 @@ public class HFileReaderV2 extends AbstractHFileReader { long memstoreTS = 0; int memstoreTSLen = 0; int lastKeyValueSize = -1; - KeyValue.KeyOnlyKeyValue keyOnlykv = new KeyValue.KeyOnlyKeyValue(); + ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue keyOnlyKv = + new ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue(); do { blockBuffer.mark(); klen = blockBuffer.getInt(); vlen = blockBuffer.getInt(); - blockBuffer.reset(); + ByteBuffer bb = blockBuffer.asSubBuffer(klen); + keyOnlyKv.setKey(bb, bb.position(), klen); + int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlyKv); if (this.reader.shouldIncludeMemstoreTS()) { if (this.reader.decodeMemstoreTS) { try { - int memstoreTSOffset = blockBuffer.arrayOffset() + blockBuffer.position() - + KEY_VALUE_LEN_SIZE + klen + vlen; - memstoreTS = Bytes.readVLong(blockBuffer.array(), memstoreTSOffset); + blockBuffer.skip(klen + vlen); + memstoreTS = blockBuffer.getVLong(); memstoreTSLen = WritableUtils.getVIntSize(memstoreTS); } catch (Exception e) { throw new RuntimeException("Error reading memstore timestamp", e); @@ -995,21 +1118,16 @@ public class HFileReaderV2 extends AbstractHFileReader { memstoreTSLen = 1; } } - - int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position() + KEY_VALUE_LEN_SIZE; - keyOnlykv.setKey(blockBuffer.array(), keyOffset, klen); - int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlykv); - + blockBuffer.reset(); if (comp == 0) { if (seekBefore) { if (lastKeyValueSize < 0) { throw new IllegalStateException("blockSeek with seekBefore " - + "at the first key of the block: key=" - + CellUtil.getCellKeyAsString(key) - + ", blockOffset=" + block.getOffset() + ", onDiskSize=" - + block.getOnDiskSizeWithHeader()); + + "at the first key of the block: key=" + CellUtil.getCellKeyAsString(key) + + ", blockOffset=" + blockHolder.getCurrentBlock().getOffset() + ", onDiskSize=" + + blockHolder.getCurrentBlock().getOnDiskSizeWithHeader()); } - blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + blockBuffer.moveBack(lastKeyValueSize); readKeyValueLen(); return 1; // non exact match. } @@ -1021,8 +1139,9 @@ public class HFileReaderV2 extends AbstractHFileReader { } return 0; // indicate exact match } else if (comp < 0) { - if (lastKeyValueSize > 0) - blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + if (lastKeyValueSize > 0) { + blockBuffer.moveBack(lastKeyValueSize); + } readKeyValueLen(); if (lastKeyValueSize == -1 && blockBuffer.position() == 0 && this.reader.trailer.getMinorVersion() >= MINOR_VERSION_WITH_FAKED_KEY) { @@ -1033,50 +1152,46 @@ public class HFileReaderV2 extends AbstractHFileReader { // The size of this key/value tuple, including key/value length fields. lastKeyValueSize = klen + vlen + memstoreTSLen + KEY_VALUE_LEN_SIZE; - blockBuffer.position(blockBuffer.position() + lastKeyValueSize); - } while (blockBuffer.remaining() > 0); + blockBuffer.skip(lastKeyValueSize); + } while (blockBuffer.hasRemaining()); // Seek to the last key we successfully read. This will happen if this is // the last key/value pair in the file, in which case the following call // to next() has to return false. - blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + blockBuffer.moveBack(lastKeyValueSize); readKeyValueLen(); return 1; // didn't exactly find it. } @Override protected ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) { - ByteBuffer buffer = curBlock.getBufferWithoutHeader(); + MultiByteBuffer buffer = curBlock.getBufferWithoutHeader(); // It is safe to manipulate this buffer because we own the buffer object. buffer.rewind(); int klen = buffer.getInt(); - buffer.getInt(); - ByteBuffer keyBuff = buffer.slice(); - keyBuff.limit(klen); - keyBuff.rewind(); - return keyBuff; + buffer.skip(Bytes.SIZEOF_INT);// Skip value len part + ByteBuffer keyBuf = buffer.asSubBuffer(klen); + keyBuf.limit(KEY_VALUE_LEN_SIZE + klen); + return keyBuf.slice(); } @Override public String getKeyString() { - return Bytes.toStringBinary(blockBuffer.array(), - blockBuffer.arrayOffset() + blockBuffer.position() - + KEY_VALUE_LEN_SIZE, currKeyLen); + return ByteBufferUtils.toStringBinary(getKey()); } @Override public String getValueString() { - return Bytes.toString(blockBuffer.array(), blockBuffer.arrayOffset() - + blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen, - currValueLen); + return ByteBufferUtils.toString(getValue()); } @Override public int compareKey(KVComparator comparator, Cell key) { - return comparator.compareOnlyKeyPortion( - key, - new KeyValue.KeyOnlyKeyValue(blockBuffer.array(), blockBuffer.arrayOffset() - + blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen)); + ByteBuffer bb = blockBuffer.asSubBuffer(currKeyLen); + return comparator.compareOnlyKeyPortion(key, + new ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue(bb, bb.position(), + currKeyLen)); + //return comparator.compareOnlyKeyPortion(key, blockBuffer, blockBuffer.position(), currKeyLen); } } @@ -1088,123 +1203,115 @@ public class HFileReaderV2 extends AbstractHFileReader { private final DataBlockEncoder.EncodedSeeker seeker; private final DataBlockEncoder dataBlockEncoder; protected final HFileContext meta; + private final HFileReaderV2 reader; - public EncodedScannerV2(HFileReaderV2 reader, boolean cacheBlocks, - boolean pread, boolean isCompaction, HFileContext meta) { + public EncodedScannerV2(HFileReaderV2 reader, boolean cacheBlocks, boolean pread, + boolean isCompaction, HFileContext meta) { super(reader, cacheBlocks, pread, isCompaction); DataBlockEncoding encoding = reader.dataBlockEncoder.getDataBlockEncoding(); dataBlockEncoder = encoding.getEncoder(); decodingCtx = dataBlockEncoder.newDataBlockDecodingContext(meta); - seeker = dataBlockEncoder.createSeeker( - reader.getComparator(), decodingCtx); + seeker = dataBlockEncoder.createSeeker(reader.getComparator(), decodingCtx); this.meta = meta; + this.reader = reader; } @Override - public boolean isSeeked(){ - return this.block != null; + public boolean isSeeked() { + return blockHolder.getCurrentBlock() != null; + } + + public void setNonSeekedState() { + this.blockHolder.reset(); } /** * Updates the current block to be the given {@link HFileBlock}. Seeks to * the the first key/value pair. - * - * @param newBlock the block to make current + * + * @param newBlock + * the block to make current * @throws CorruptHFileException */ - private void updateCurrentBlock(HFileBlock newBlock) throws CorruptHFileException { - block = newBlock; - + @Override + protected void updateCurrentBlock(HFileBlock newBlock) throws CorruptHFileException { // sanity checks - if (block.getBlockType() != BlockType.ENCODED_DATA) { - throw new IllegalStateException( - "EncodedScanner works only on encoded data blocks"); + if (newBlock.getBlockType() != BlockType.ENCODED_DATA) { + throw new IllegalStateException("EncodedScanner works only on encoded data blocks"); } - short dataBlockEncoderId = block.getDataBlockEncodingId(); + short dataBlockEncoderId = newBlock.getDataBlockEncodingId(); if (!DataBlockEncoding.isCorrectEncoder(dataBlockEncoder, dataBlockEncoderId)) { String encoderCls = dataBlockEncoder.getClass().getName(); throw new CorruptHFileException("Encoder " + encoderCls - + " doesn't support data block encoding " - + DataBlockEncoding.getNameFromId(dataBlockEncoderId)); + + " doesn't support data block encoding " + + DataBlockEncoding.getNameFromId(dataBlockEncoderId)); } - seeker.setCurrentBuffer(getEncodedBuffer(newBlock)); + blockHolder.updateCurrentBlock(newBlock); + MultiByteBuffer encodedBuffer = getEncodedBuffer(newBlock); + // TODO : We need to deal with MBB only? If we don't do asSubBuffer here + // then + // in the ClonedSeekerState we need to convert the MBB to BB in + // getXXXBuffer methods + seeker.setCurrentBuffer(encodedBuffer.asSubBuffer(encodedBuffer.capacity())); blockFetches++; // Reset the next indexed key this.nextIndexedKey = null; } - private ByteBuffer getEncodedBuffer(HFileBlock newBlock) { - ByteBuffer origBlock = newBlock.getBufferReadOnly(); - ByteBuffer encodedBlock = ByteBuffer.wrap(origBlock.array(), - origBlock.arrayOffset() + newBlock.headerSize() + - DataBlockEncoding.ID_SIZE, - newBlock.getUncompressedSizeWithoutHeader() - - DataBlockEncoding.ID_SIZE).slice(); - return encodedBlock; + private MultiByteBuffer getEncodedBuffer(HFileBlock newBlock) { + MultiByteBuffer origBlock = newBlock.getBufferReadOnly(); + int pos = newBlock.headerSize() + DataBlockEncoding.ID_SIZE; + origBlock.position(pos); + origBlock + .limit(pos + newBlock.getUncompressedSizeWithoutHeader() - DataBlockEncoding.ID_SIZE); + return origBlock.slice(); } @Override - public boolean seekTo() throws IOException { - if (reader == null) { - return false; - } - - if (reader.getTrailer().getEntryCount() == 0) { - // No data blocks. - return false; - } - - long firstDataBlockOffset = - reader.getTrailer().getFirstDataBlockOffset(); - if (block != null && block.getOffset() == firstDataBlockOffset) { - seeker.rewind(); - return true; - } - - block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread, - isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding()); - if (block.getOffset() < 0) { - throw new IOException("Invalid block offset: " + block.getOffset()); - } - updateCurrentBlock(block); + protected boolean processFirstDataBlock() throws IOException { + seeker.rewind(); return true; } @Override public boolean next() throws IOException { - boolean isValid = seeker.next(); - if (!isValid) { - block = readNextDataBlock(); - isValid = block != null; - if (isValid) { - updateCurrentBlock(block); + try { + boolean isValid = seeker.next(); + if (!isValid) { + HFileBlock newBlock = readNextDataBlock(); + isValid = newBlock != null; + if (isValid) { + updateCurrentBlock(newBlock); + } else { + setNonSeekedState(); + } } + return isValid; + } catch (Exception e) { + this.reader.returnBlock(this.blockHolder.getCurrentBlock()); + throw e; } - return isValid; } @Override public ByteBuffer getKey() { assertValidSeek(); + // TODO return seeker.getKeyDeepCopy(); } @Override - public int compareKey(KVComparator comparator, byte[] key, int offset, int length) { - return seeker.compareKey(comparator, key, offset, length); - } - - @Override public ByteBuffer getValue() { assertValidSeek(); + // TODO return seeker.getValueShallowCopy(); } @Override public Cell getKeyValue() { - if (block == null) { + if (blockHolder.getCurrentBlock() == null) { return null; } return seeker.getKeyValue(); @@ -1212,20 +1319,16 @@ public class HFileReaderV2 extends AbstractHFileReader { @Override public String getKeyString() { - ByteBuffer keyBuffer = getKey(); - return Bytes.toStringBinary(keyBuffer.array(), - keyBuffer.arrayOffset(), keyBuffer.limit()); + return Bytes.toStringBinary(getKey()); } @Override public String getValueString() { - ByteBuffer valueBuffer = getValue(); - return Bytes.toStringBinary(valueBuffer.array(), - valueBuffer.arrayOffset(), valueBuffer.limit()); + return Bytes.toStringBinary(getValue()); } private void assertValidSeek() { - if (block == null) { + if (blockHolder.getCurrentBlock() == null) { throw new NotSeekedException(); } } @@ -1238,7 +1341,8 @@ public class HFileReaderV2 extends AbstractHFileReader { @Override protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey, boolean rewind, Cell key, boolean seekBefore) throws IOException { - if (block == null || block.getOffset() != seekToBlock.getOffset()) { + if (blockHolder.getCurrentBlock() == null + || blockHolder.getCurrentBlock().getOffset() != seekToBlock.getOffset()) { updateCurrentBlock(seekToBlock); } else if (rewind) { seeker.rewind(); @@ -1254,8 +1358,8 @@ public class HFileReaderV2 extends AbstractHFileReader { } /** - * Returns a buffer with the Bloom filter metadata. The caller takes - * ownership of the buffer. + * Returns a buffer with the Bloom filter metadata. The caller takes ownership + * of the buffer. */ @Override public DataInput getGeneralBloomFilterMetadata() throws IOException { @@ -1267,12 +1371,10 @@ public class HFileReaderV2 extends AbstractHFileReader { return this.getBloomFilterMetadata(BlockType.DELETE_FAMILY_BLOOM_META); } - private DataInput getBloomFilterMetadata(BlockType blockType) - throws IOException { - if (blockType != BlockType.GENERAL_BLOOM_META && - blockType != BlockType.DELETE_FAMILY_BLOOM_META) { - throw new RuntimeException("Block Type: " + blockType.toString() + - " is not supported") ; + private DataInput getBloomFilterMetadata(BlockType blockType) throws IOException { + if (blockType != BlockType.GENERAL_BLOOM_META + && blockType != BlockType.DELETE_FAMILY_BLOOM_META) { + throw new RuntimeException("Block Type: " + blockType.toString() + " is not supported"); } for (HFileBlock b : loadOnOpenBlocks) @@ -1287,16 +1389,13 @@ public class HFileReaderV2 extends AbstractHFileReader { } /** - * Validates that the minor version is within acceptable limits. - * Otherwise throws an Runtime exception + * Validates that the minor version is within acceptable limits. Otherwise + * throws an Runtime exception */ private void validateMinorVersion(Path path, int minorVersion) { - if (minorVersion < MIN_MINOR_VERSION || - minorVersion > MAX_MINOR_VERSION) { - String msg = "Minor version for path " + path + - " is expected to be between " + - MIN_MINOR_VERSION + " and " + MAX_MINOR_VERSION + - " but is found to be " + minorVersion; + if (minorVersion < MIN_MINOR_VERSION || minorVersion > MAX_MINOR_VERSION) { + String msg = "Minor version for path " + path + " is expected to be between " + + MIN_MINOR_VERSION + " and " + MAX_MINOR_VERSION + " but is found to be " + minorVersion; LOG.error(msg); throw new RuntimeException(msg); } @@ -1313,8 +1412,8 @@ public class HFileReaderV2 extends AbstractHFileReader { } /** - * Returns false if block prefetching was requested for this file and has - * not completed, true otherwise + * Returns false if block prefetching was requested for this file and has not + * completed, true otherwise */ @VisibleForTesting boolean prefetchComplete() { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java index b28d8c1..8f0206c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java @@ -18,18 +18,19 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.IOException; +import java.nio.ByteBuffer; import java.security.Key; import java.security.KeyException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.crypto.Cipher; @@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.WritableUtils; @@ -91,7 +91,8 @@ public class HFileReaderV3 extends HFileReaderV2 { HFileContextBuilder builder = new HFileContextBuilder() .withIncludesMvcc(this.includesMemstoreTS) .withHBaseCheckSum(true) - .withCompression(this.compressAlgo); + .withCompression(this.compressAlgo) + .withPathName(path.getName()); // Check for any key material available byte[] keyBytes = trailer.getEncryptionKey(); @@ -195,15 +196,16 @@ public class HFileReaderV3 extends HFileReaderV2 { } @Override - protected int getNextCellStartPosition() { - int nextKvPos = super.getNextCellStartPosition(); + protected int getCurCellSize() { + int curCellSize = super.getCurCellSize(); if (reader.hfileContext.isIncludesTags()) { - nextKvPos += Bytes.SIZEOF_SHORT + currTagsLen; + curCellSize += Bytes.SIZEOF_SHORT + currTagsLen; } - return nextKvPos; + return curCellSize; } protected void readKeyValueLen() { + // TODO : Avoid reading again. Is it possible to cache the prev KV's value? some how blockBuffer.mark(); currKeyLen = blockBuffer.getInt(); currValueLen = blockBuffer.getInt(); @@ -211,19 +213,19 @@ public class HFileReaderV3 extends HFileReaderV2 { || currValueLen > blockBuffer.limit()) { throw new IllegalStateException("Invalid currKeyLen " + currKeyLen + " or currValueLen " + currValueLen + ". Block offset: " - + block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + + blockHolder.getCurrentBlock().getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + blockBuffer.position() + " (without header)."); } - ByteBufferUtils.skip(blockBuffer, currKeyLen + currValueLen); + blockBuffer.skip(currKeyLen + currValueLen); if (reader.hfileContext.isIncludesTags()) { // Read short as unsigned, high byte first currTagsLen = ((blockBuffer.get() & 0xff) << 8) ^ (blockBuffer.get() & 0xff); if (currTagsLen < 0 || currTagsLen > blockBuffer.limit()) { throw new IllegalStateException("Invalid currTagsLen " + currTagsLen + ". Block offset: " - + block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + + blockHolder.getCurrentBlock().getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + blockBuffer.position() + " (without header)."); } - ByteBufferUtils.skip(blockBuffer, currTagsLen); + blockBuffer.skip(currTagsLen); } readMvccVersion(); blockBuffer.reset(); @@ -250,7 +252,8 @@ public class HFileReaderV3 extends HFileReaderV2 { long memstoreTS = 0; int memstoreTSLen = 0; int lastKeyValueSize = -1; - KeyValue.KeyOnlyKeyValue keyOnlyKv = new KeyValue.KeyOnlyKeyValue(); + ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue keyOnlyKv = + new ByteBufferBackedKeyValue.ByteBufferBackedKeyOnlyKeyValue(); do { blockBuffer.mark(); klen = blockBuffer.getInt(); @@ -259,25 +262,27 @@ public class HFileReaderV3 extends HFileReaderV2 { || vlen > blockBuffer.limit()) { throw new IllegalStateException("Invalid klen " + klen + " or vlen " + vlen + ". Block offset: " - + block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + + blockHolder.getCurrentBlock().getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + blockBuffer.position() + " (without header)."); } - ByteBufferUtils.skip(blockBuffer, klen + vlen); + ByteBuffer bb = blockBuffer.asSubBuffer(klen); + keyOnlyKv.setKey(bb, bb.position(), klen); + int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlyKv); + blockBuffer.skip(klen + vlen); if (reader.hfileContext.isIncludesTags()) { // Read short as unsigned, high byte first tlen = ((blockBuffer.get() & 0xff) << 8) ^ (blockBuffer.get() & 0xff); if (tlen < 0 || tlen > blockBuffer.limit()) { throw new IllegalStateException("Invalid tlen " + tlen + ". Block offset: " - + block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + + blockHolder.getCurrentBlock().getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + blockBuffer.position() + " (without header)."); } - ByteBufferUtils.skip(blockBuffer, tlen); + blockBuffer.skip(tlen); } if (this.reader.shouldIncludeMemstoreTS()) { if (this.reader.decodeMemstoreTS) { try { - memstoreTS = Bytes.readVLong(blockBuffer.array(), blockBuffer.arrayOffset() - + blockBuffer.position()); + memstoreTS = blockBuffer.getVLong(); memstoreTSLen = WritableUtils.getVIntSize(memstoreTS); } catch (Exception e) { throw new RuntimeException("Error reading memstore timestamp", e); @@ -288,20 +293,16 @@ public class HFileReaderV3 extends HFileReaderV2 { } } blockBuffer.reset(); - int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position() + (Bytes.SIZEOF_INT * 2); - keyOnlyKv.setKey(blockBuffer.array(), keyOffset, klen); - int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlyKv); - if (comp == 0) { if (seekBefore) { if (lastKeyValueSize < 0) { throw new IllegalStateException("blockSeek with seekBefore " + "at the first key of the block: key=" + CellUtil.getCellKeyAsString(key) - + ", blockOffset=" + block.getOffset() + ", onDiskSize=" - + block.getOnDiskSizeWithHeader()); + + ", blockOffset=" + blockHolder.getCurrentBlock().getOffset() + ", onDiskSize=" + + blockHolder.getCurrentBlock().getOnDiskSizeWithHeader()); } - blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + blockBuffer.moveBack(lastKeyValueSize); readKeyValueLen(); return 1; // non exact match. } @@ -314,8 +315,9 @@ public class HFileReaderV3 extends HFileReaderV2 { } return 0; // indicate exact match } else if (comp < 0) { - if (lastKeyValueSize > 0) - blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + if (lastKeyValueSize > 0) { + blockBuffer.moveBack(lastKeyValueSize); + } readKeyValueLen(); if (lastKeyValueSize == -1 && blockBuffer.position() == 0) { return HConstants.INDEX_KEY_MAGIC; @@ -329,13 +331,13 @@ public class HFileReaderV3 extends HFileReaderV2 { if (reader.hfileContext.isIncludesTags()) { lastKeyValueSize += tlen + Bytes.SIZEOF_SHORT; } - blockBuffer.position(blockBuffer.position() + lastKeyValueSize); - } while (blockBuffer.remaining() > 0); + blockBuffer.skip(lastKeyValueSize); + } while (blockBuffer.hasRemaining()); // Seek to the last key we successfully read. This will happen if this is // the last key/value pair in the file, in which case the following call // to next() has to return false. - blockBuffer.position(blockBuffer.position() - lastKeyValueSize); + blockBuffer.moveBack(lastKeyValueSize); readKeyValueLen(); return 1; // didn't exactly find it. } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java index deaa2c0..c8d2e54 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java @@ -21,8 +21,9 @@ package org.apache.hadoop.hbase.io.hfile; import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.classification.InterfaceAudience; /** * A scanner allows you to position yourself within a HFile and @@ -161,4 +162,5 @@ public interface HFileScanner { * @return the next key in the index (the key to seek to the next block) */ Cell getNextIndexedKey(); + void close(); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java index 28c4655..634b433 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java @@ -142,7 +142,6 @@ public class HFileWriterV2 extends AbstractHFileWriter { protected void checkBlockBoundary() throws IOException { if (fsBlockWriter.blockSizeWritten() < hFileContext.getBlocksize()) return; - finishBlock(); writeInlineBlocks(false); newBlock(); @@ -253,12 +252,9 @@ public class HFileWriterV2 extends AbstractHFileWriter { */ @Override public void append(final Cell cell) throws IOException { - byte[] value = cell.getValueArray(); - int voffset = cell.getValueOffset(); - int vlength = cell.getValueLength(); // checkKey uses comparator to check we are writing in order. boolean dupKey = checkKey(cell); - checkValue(value, voffset, vlength); + checkValue(cell); if (!dupKey) { checkBlockBoundary(); } @@ -270,7 +266,7 @@ public class HFileWriterV2 extends AbstractHFileWriter { fsBlockWriter.write(cell); totalKeyLength += CellUtil.estimatedSerializedSizeOfKey(cell); - totalValueLength += vlength; + totalValueLength += cell.getValueLength(); // Are we the first key in this block? if (firstCellInBlock == null) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java index 82df5f7..bc4f611 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java @@ -34,13 +34,14 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; -import com.google.common.base.Objects; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.HFileBlock.CacheType; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; @@ -49,6 +50,7 @@ import org.apache.hadoop.util.StringUtils; import org.codehaus.jackson.annotate.JsonIgnoreProperties; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; import com.google.common.util.concurrent.ThreadFactoryBuilder; /** @@ -328,7 +330,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { LruCachedBlock cb = map.get(cacheKey); if (cb != null) { // compare the contents, if they are not equal, we are in big trouble - if (compare(buf, cb.getBuffer()) != 0) { + if (compare(buf, cb.getBuffer()) == false) { throw new RuntimeException("Cached block contents differ, which should not have happened." + "cacheKey:" + cacheKey); } @@ -369,12 +371,15 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { } } - private int compare(Cacheable left, Cacheable right) { + private boolean compare(Cacheable left, Cacheable right) { + if (left instanceof HFileBlock && right instanceof HFileBlock) { + return left.equals(right); + } ByteBuffer l = ByteBuffer.allocate(left.getSerializedLength()); left.serialize(l); ByteBuffer r = ByteBuffer.allocate(right.getSerializedLength()); right.serialize(r); - return Bytes.compareTo(l.array(), l.arrayOffset(), l.limit(), + return Bytes.equals(l.array(), l.arrayOffset(), l.limit(), r.array(), r.arrayOffset(), r.limit()); } @@ -426,6 +431,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { } if (updateCacheMetrics) stats.hit(caching); cb.access(count.incrementAndGet()); + if (cb.getBuffer() instanceof HFileBlock) { + ((HFileBlock) cb.getBuffer()).setCacheType(CacheType.L1_CACHED); + } return cb.getBuffer(); } @@ -1075,4 +1083,17 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { public BlockCache[] getBlockCaches() { return null; } + + @Override + public boolean returnBlock(BlockCacheKey cacheKey, HFileBlock block) { + // If the L1 and L2 are not in combined mode but L2 is just made as victim for L1, we will have + // to return block to L2 cache when the same was obtained from it + if (block.getCacheType() == CacheType.L2_CACHED) { + return this.victimHandler.returnBlock(cacheKey, block); + } + // When the block was served by L1 cache, do nothing + assert block.getCacheType() == CacheType.L1_CACHED; + // Do nothing + return true; + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java index f75f6e9..a27a402 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext; import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext; import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext; import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.io.WritableUtils; /** @@ -50,23 +51,27 @@ public class NoOpDataBlockEncoder implements HFileDataBlockEncoder { int klength = KeyValueUtil.keyLength(cell); int vlength = cell.getValueLength(); + int encodedKvSize = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; + int tagsLength = cell.getTagsLength(); + if (encodingCtx.getHFileContext().isIncludesTags()) { + encodedKvSize += tagsLength + KeyValue.TAGS_LENGTH_SIZE; + } + if (encodingCtx.getHFileContext().isIncludesMvcc()) { + encodedKvSize += WritableUtils.getVIntSize(cell.getSequenceId()); + } out.writeInt(klength); out.writeInt(vlength); CellUtil.writeFlatKey(cell, out); - out.write(cell.getValueArray(), cell.getValueOffset(), vlength); - int encodedKvSize = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; + CellUtil.writeValue(out, cell, vlength); // Write the additional tag into the stream if (encodingCtx.getHFileContext().isIncludesTags()) { - int tagsLength = cell.getTagsLength(); out.writeShort(tagsLength); if (tagsLength > 0) { - out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength); + CellUtil.writeTags(out, cell, tagsLength); } - encodedKvSize += tagsLength + KeyValue.TAGS_LENGTH_SIZE; } if (encodingCtx.getHFileContext().isIncludesMvcc()) { WritableUtils.writeVLong(out, cell.getSequenceId()); - encodedKvSize += WritableUtils.getVIntSize(cell.getSequenceId()); } return encodedKvSize; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 7dda0e6..c4515ee 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -42,6 +42,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -61,16 +62,19 @@ import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer; import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager; import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.HFileBlock; +import org.apache.hadoop.hbase.io.hfile.HFileBlock.CacheType; import org.apache.hadoop.hbase.util.ConcurrentIndex; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.util.IdLock; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.util.StringUtils; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ThreadFactoryBuilder; + /** * BucketCache uses {@link BucketAllocator} to allocate/free blocks, and uses * {@link BucketCache#ramCache} and {@link BucketCache#backingMap} in order to @@ -356,6 +360,8 @@ public class BucketCache implements BlockCache, HeapSize { * Stuff the entry into the RAM cache so it can get drained to the * persistent store */ + // all the blocks that get cached will create a new RAMQueueEntry. So + // the reset will happen automatically RAMQueueEntry re = new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory); ramCache.put(cacheKey, re); @@ -410,8 +416,8 @@ public class BucketCache implements BlockCache, HeapSize { lockEntry = offsetLock.getLockEntry(bucketEntry.offset()); if (bucketEntry.equals(backingMap.get(key))) { int len = bucketEntry.getLength(); - ByteBuffer bb = ByteBuffer.allocate(len); - int lenRead = ioEngine.read(bb, bucketEntry.offset()); + MultiByteBuffer bb = ioEngine.read(bucketEntry.offset(), len); + int lenRead = bb.limit(); if (lenRead != len) { throw new RuntimeException("Only " + lenRead + " bytes read, " + len + " expected"); } @@ -423,6 +429,10 @@ public class BucketCache implements BlockCache, HeapSize { cacheStats.hit(caching); cacheStats.ioHit(timeTaken); } + if (cachedBlock instanceof HFileBlock) { + ((HFileBlock) cachedBlock).setCacheType(CacheType.L2_CACHED); + bucketEntry.blockRefCount.incrementAndGet(); + } bucketEntry.access(accessCount.incrementAndGet()); if (this.ioErrorStartTime > 0) { ioErrorStartTime = -1; @@ -444,7 +454,12 @@ public class BucketCache implements BlockCache, HeapSize { @Override public boolean evictBlock(BlockCacheKey cacheKey) { - if (!cacheEnabled) return false; + return evictBlock(cacheKey, false); + } + + public boolean evictBlock(BlockCacheKey cacheKey, boolean withInUseCheck) { + if (!cacheEnabled) + return false; RAMQueueEntry removedBlock = ramCache.remove(cacheKey); if (removedBlock != null) { this.blockNumber.decrementAndGet(); @@ -462,14 +477,20 @@ public class BucketCache implements BlockCache, HeapSize { IdLock.Entry lockEntry = null; try { lockEntry = offsetLock.getLockEntry(bucketEntry.offset()); - if (bucketEntry.equals(backingMap.remove(cacheKey))) { - bucketAllocator.freeBlock(bucketEntry.offset()); - realCacheSize.addAndGet(-1 * bucketEntry.getLength()); - blocksByHFile.remove(cacheKey.getHfileName(), cacheKey); - if (removedBlock == null) { - this.blockNumber.decrementAndGet(); + int refCount = bucketEntry.blockRefCount.get(); + if (!withInUseCheck || refCount == 0) { + if (bucketEntry.equals(backingMap.remove(cacheKey))) { + bucketAllocator.freeBlock(bucketEntry.offset()); + realCacheSize.addAndGet(-1 * bucketEntry.getLength()); + blocksByHFile.remove(cacheKey.getHfileName(), cacheKey); + if (removedBlock == null) { + this.blockNumber.decrementAndGet(); + } + } else { + return false; } } else { + LOG.info("This block is still referred by " + refCount + " readers. Can not be freed now"); return false; } } catch (IOException ie) { @@ -1063,6 +1084,7 @@ public class BucketCache implements BlockCache, HeapSize { byte deserialiserIndex; private volatile long accessTime; private BlockPriority priority; + private AtomicInteger blockRefCount = new AtomicInteger(0); /** * Time this block was cached. Presumes we are created just before we are added to the cache. */ @@ -1165,14 +1187,21 @@ public class BucketCache implements BlockCache, HeapSize { public long free(long toFree) { Map.Entry entry; long freedBytes = 0; - while ((entry = queue.pollLast()) != null) { - evictBlock(entry.getKey()); - freedBytes += entry.getValue().getLength(); - if (freedBytes >= toFree) { + while (true) { + // We should change this to peekLast and only on finding that the eviction has + // happened should we remove it from the queue? + entry = queue.pollLast(); + if (entry != null) { + if (evictBlock(entry.getKey(), true)) { + freedBytes += entry.getValue().getLength(); + if (freedBytes >= toFree) { + return freedBytes; + } + } + } else { return freedBytes; } } - return freedBytes; } public long overflow() { @@ -1208,7 +1237,7 @@ public class BucketCache implements BlockCache, HeapSize { private boolean inMemory; public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessTime, - boolean inMemory) { + boolean inMemory ) { this.key = bck; this.data = data; this.accessTime = accessTime; @@ -1236,12 +1265,13 @@ public class BucketCache implements BlockCache, HeapSize { // This cacheable thing can't be serialized... if (len == 0) return null; long offset = bucketAllocator.allocateBlock(len); + // Every time a new bucketentry is created BucketEntry bucketEntry = new BucketEntry(offset, len, accessTime, inMemory); bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap); try { if (data instanceof HFileBlock) { HFileBlock block = (HFileBlock) data; - ByteBuffer sliceBuf = block.getBufferReadOnlyWithHeader(); + MultiByteBuffer sliceBuf = block.getBufferReadOnlyWithHeader(); sliceBuf.rewind(); assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE || len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE; @@ -1368,6 +1398,18 @@ public class BucketCache implements BlockCache, HeapSize { } @Override + public boolean returnBlock(BlockCacheKey cacheKey, HFileBlock block) { + assert block.getCacheType() == CacheType.L2_CACHED; + // When the block was served from RAMQueueEntry, it wont set the CacheType as L2. It is still on + // its way to get into L2 cache. So still it is like NOT_FROM_CACHE. + BucketEntry bucketEntry = backingMap.get(cacheKey); + if (bucketEntry != null) { + bucketEntry.blockRefCount.decrementAndGet(); + return true; + } + return false; + } + @Override public BlockCache[] getBlockCaches() { return null; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java index de10667..61f796f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteBufferArray; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * IO engine that stores data in memory using an array of ByteBuffers @@ -78,6 +79,11 @@ public class ByteBufferIOEngine implements IOEngine { dstBuffer.arrayOffset()); } + @Override + public MultiByteBuffer read(long offset, int len) throws IOException { + return bufferArray.subArray(offset, len); + } + /** * Transfers data from the given byte buffer to the buffer array * @param srcBuffer the given byte buffer from which bytes are to be read @@ -92,6 +98,15 @@ public class ByteBufferIOEngine implements IOEngine { srcBuffer.arrayOffset()); } + @Override + public void write(MultiByteBuffer srcBuffer, long offset) throws IOException { + // When caching block into BucketCache there will be single buffer backing for this HFileBlock. + // This will work for now. But from the DFS itself if we get DBB then this may not hold true. + assert srcBuffer.hasArray(); + bufferArray.putMultiple(offset, srcBuffer.remaining(), srcBuffer.array(), + srcBuffer.arrayOffset()); + } + /** * No operation for the sync in the memory IO engine */ diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java index b6954bb..fdc5d03 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CachedEntryQueue.java @@ -105,7 +105,7 @@ public class CachedEntryQueue { } /** - * @return The last element in this queue, or {@code null} if the queue is + * @return Removes and returns last element in this queue, or {@code null} if the queue is * empty. */ public Map.Entry pollLast() { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java index 7b6b25f..8d60db3 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java @@ -26,6 +26,7 @@ import java.nio.channels.FileChannel; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.util.StringUtils; /** @@ -89,6 +90,15 @@ public class FileIOEngine implements IOEngine { return fileChannel.read(dstBuffer, offset); } + + @Override + public MultiByteBuffer read(long offset, int len) throws IOException { + ByteBuffer dstBuffer = ByteBuffer.allocate(len); + int read = read(dstBuffer, offset); + dstBuffer.limit(read); + return new MultiByteBuffer(dstBuffer); + } + /** * Transfers data from the given byte buffer to file * @param srcBuffer the given byte buffer from which bytes are to be read @@ -100,6 +110,14 @@ public class FileIOEngine implements IOEngine { fileChannel.write(srcBuffer, offset); } + @Override + public void write(MultiByteBuffer srcBuffer, long offset) throws IOException { + // When caching block into BucketCache there will be single buffer backing for this HFileBlock. + assert srcBuffer.hasArray(); + fileChannel.write( + ByteBuffer.wrap(srcBuffer.array(), srcBuffer.arrayOffset(), srcBuffer.remaining()), offset); + } + /** * Sync the data to file after writing * @throws IOException diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/IOEngine.java hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/IOEngine.java index 430c5af..f665d2c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/IOEngine.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/IOEngine.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.MultiByteBuffer; /** * A class implementing IOEngine interface supports data services for @@ -43,6 +44,8 @@ public interface IOEngine { */ int read(ByteBuffer dstBuffer, long offset) throws IOException; + MultiByteBuffer read(long offset, int len) throws IOException; + /** * Transfers data from the given byte buffer to IOEngine * @param srcBuffer the given byte buffer from which bytes are to be read @@ -51,6 +54,8 @@ public interface IOEngine { * @throws IOException */ void write(ByteBuffer srcBuffer, long offset) throws IOException; + + void write(MultiByteBuffer srcBuffer, long offset) throws IOException; /** * Sync the data to IOEngine after writing diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 9458693..d6470fc 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -67,8 +67,6 @@ import javax.security.sasl.SaslServer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -78,9 +76,12 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Operation; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.exceptions.RegionMovedException; +import org.apache.hadoop.hbase.io.BoundedByteBufferPool; import org.apache.hadoop.hbase.io.ByteBufferOutputStream; import org.apache.hadoop.hbase.io.BoundedByteBufferPool; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; @@ -267,10 +268,9 @@ public class RpcServer implements RpcServerInterface { private final RpcScheduler scheduler; private UserProvider userProvider; - + private final BoundedByteBufferPool reservoir; - /** * Datastructure that holds all necessary to a method invocation and then afterward, carries * the result. @@ -328,7 +328,7 @@ public class RpcServer implements RpcServerInterface { reservoir.putBuffer(this.cellBlock); this.cellBlock = null; } - this.connection.decRpcCount(); // Say that we're done with this call. + this.connection.decRpcCount(); } @Override @@ -393,11 +393,23 @@ public class RpcServer implements RpcServerInterface { // Set the exception as the result of the method invocation. headerBuilder.setException(exceptionBuilder.build()); } + // Get a bb from the reservoir and pass it to buildCellBlock. What comes + // back will be the + // passed in reservoir bb or a resized one that we should instead add + // back to the reservoir + // when done. Keep reference so can add it back to the reservoir when + // finished. This is + // hacky and the hack is not contained but benefits are high when we can + // avoid a big buffer + // allocation on each rpc. // Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the // reservoir when finished. This is hacky and the hack is not contained but benefits are // high when we can avoid a big buffer allocation on each rpc. + // Note : On the server side we could always go with offheap buffers. + // Internally the SocketChannel would avoid a copy from onheap to offheap + // BBs if we go with offheap buffers here. this.cellBlock = ipcUtil.buildCellBlock(this.connection.codec, - this.connection.compressionCodec, cells, reservoir); + this.connection.compressionCodec, cells, true, reservoir); if (this.cellBlock != null) { CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder(); // Presumes the cellBlock bytebuffer has been flipped so limit has total size in it. @@ -1072,7 +1084,7 @@ public class RpcServer implements RpcServerInterface { } if (!call.response.hasRemaining()) { - call.done(); + call.done(); // Say that we're done with this call. return true; } else { return false; // Socket can't take more, we will have to come back. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java index b3981a1..d95e745 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TextSortReducer.java @@ -27,6 +27,7 @@ import java.util.TreeSet; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -174,7 +175,7 @@ public class TextSortReducer extends // Add TTL directly to the KV so we can vary them when packing more than one KV // into puts if (ttl > 0) { - tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl))); + tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl))); } Cell cell = this.kvCreator.create(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(), parser.getFamily(i), 0, parser.getFamily(i).length, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java index 270de75..bff0cc4 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Tag; @@ -215,7 +216,7 @@ extends Mapper // Add TTL directly to the KV so we can vary them when packing more than one KV // into puts if (ttl > 0) { - tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl))); + tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl))); } cell = this.kvCreator.create(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(), parser.getFamily(i), 0, parser.getFamily(i).length, parser.getQualifier(i), 0, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java index 71ea1bd..b917ae9 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnCount.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; */ @InterfaceAudience.Private public class ColumnCount { - private final byte [] bytes; + private final byte[] buffer; private final int offset; private final int length; private int count; @@ -36,7 +36,7 @@ public class ColumnCount { * Constructor * @param column the qualifier to count the versions for */ - public ColumnCount(byte [] column) { + public ColumnCount(byte[] column) { this(column, 0); } @@ -45,7 +45,7 @@ public class ColumnCount { * @param column the qualifier to count the versions for * @param count initial count */ - public ColumnCount(byte [] column, int count) { + public ColumnCount(byte[] column, int count) { this(column, 0, column.length, count); } @@ -56,8 +56,8 @@ public class ColumnCount { * @param length of the qualifier * @param count initial count */ - public ColumnCount(byte [] column, int offset, int length, int count) { - this.bytes = column; + public ColumnCount(byte[] column, int offset, int length, int count) { + this.buffer = column; this.offset = offset; this.length = length; this.count = count; @@ -66,8 +66,8 @@ public class ColumnCount { /** * @return the buffer */ - public byte [] getBuffer(){ - return this.bytes; + public byte[] getBuffer() { + return this.buffer; } /** diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java index 8568cfc..d78eeed 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode; @@ -64,9 +65,9 @@ public interface ColumnTracker { * @throws IOException in case there is an internal consistency problem caused by a data * corruption. */ - ScanQueryMatcher.MatchCode checkColumn(byte[] bytes, int offset, int length, byte type) + ScanQueryMatcher.MatchCode checkColumn(Cell columnCell, int offset, int length, byte type) throws IOException; - + /** * Keeps track of the number of versions for the columns asked for. It assumes that the user has * already checked if the keyvalue needs to be included by calling the @@ -84,8 +85,9 @@ public interface ColumnTracker { * @throws IOException in case there is an internal consistency problem caused by a data * corruption. */ - ScanQueryMatcher.MatchCode checkVersions(byte[] bytes, int offset, int length, long ttl, + ScanQueryMatcher.MatchCode checkVersions(Cell columnCell, int offset, int length, long ttl, byte type, boolean ignoreCount) throws IOException; + /** * Resets the Matcher */ @@ -114,9 +116,9 @@ public interface ColumnTracker { * Retrieve the MatchCode for the next row or column */ MatchCode getNextRowOrNextColumn( - byte[] bytes, int offset, int qualLength + Cell columnCell, int offset, int qualLength ); - + /** * Give the tracker a chance to declare it's done based on only the timestamp * to allow an early out. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java index 3da0c0b..7031d35 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java @@ -275,6 +275,8 @@ public class DefaultMemStore implements MemStore { return cell; } assert alloc.getBytes() != null; + // Here we can assume that this kv can be represented as BRBackedCell but the BR + // will not be a DBB KeyValueUtil.appendToByteArray(cell, alloc.getBytes(), alloc.getOffset()); KeyValue newKv = new KeyValue(alloc.getBytes(), alloc.getOffset(), len); newKv.setSequenceId(cell.getSequenceId()); @@ -959,8 +961,7 @@ public class DefaultMemStore implements MemStore { */ @Override public synchronized boolean seekToPreviousRow(Cell key) { - Cell firstKeyOnRow = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(), - key.getRowLength()); + Cell firstKeyOnRow = CellUtil.createFirstOnRow(key); SortedSet cellHead = cellSetAtCreation.headSet(firstKeyOnRow); Cell cellSetBeforeRow = cellHead.isEmpty() ? null : cellHead.last(); SortedSet snapshotHead = snapshotAtCreation @@ -972,6 +973,7 @@ public class DefaultMemStore implements MemStore { theNext = null; return false; } + // TODO : hasArray check would be needed here Cell firstKeyOnPreviousRow = KeyValueUtil.createFirstOnRow(lastCellBeforeRow.getRowArray(), lastCellBeforeRow.getRowOffset(), lastCellBeforeRow.getRowLength()); this.stopSkippingCellsIfNextRow = true; @@ -994,6 +996,7 @@ public class DefaultMemStore implements MemStore { if (higherCell == null) { return false; } + assert higherCell.hasArray(); Cell firstCellOnLastRow = KeyValueUtil.createFirstOnRow(higherCell.getRowArray(), higherCell.getRowOffset(), higherCell.getRowLength()); if (seek(firstCellOnLastRow)) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java index b779e22..b7be591 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java @@ -21,10 +21,12 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.NavigableSet; -import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; /** @@ -105,7 +107,7 @@ public class ExplicitColumnTracker implements ColumnTracker { * {@inheritDoc} */ @Override - public ScanQueryMatcher.MatchCode checkColumn(byte [] bytes, int offset, + public ScanQueryMatcher.MatchCode checkColumn(Cell columnCell, int offset, int length, byte type) { // delete markers should never be passed to an // *Explicit*ColumnTracker @@ -122,8 +124,14 @@ public class ExplicitColumnTracker implements ColumnTracker { } // Compare specific column to current column - int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), - column.getLength(), bytes, offset, length); + int ret; + if (columnCell.hasArray()) { + ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), column.getLength(), + columnCell.getQualifierArray(), offset, length); + } else { + ret = ByteBufferUtils.compareTo(column.getBuffer(), column.getOffset(), column.getLength(), + columnCell.getQualifierBuffer(), offset, length); + } // Column Matches. Return include code. The caller would call checkVersions // to limit the number of versions. @@ -156,7 +164,7 @@ public class ExplicitColumnTracker implements ColumnTracker { } @Override - public ScanQueryMatcher.MatchCode checkVersions(byte[] bytes, int offset, int length, + public ScanQueryMatcher.MatchCode checkVersions(Cell columnCell, int offset, int length, long timestamp, byte type, boolean ignoreCount) throws IOException { assert !CellUtil.isDelete(type); if (ignoreCount) return ScanQueryMatcher.MatchCode.INCLUDE; @@ -215,14 +223,20 @@ public class ExplicitColumnTracker implements ColumnTracker { * this column. We may get this information from external filters or * timestamp range and we then need to indicate this information to * tracker. It is required only in case of ExplicitColumnTracker. - * @param bytes + * @param columnCell * @param offset * @param length */ - public void doneWithColumn(byte [] bytes, int offset, int length) { + public void doneWithColumn(Cell columnCell, int offset, int length) { while (this.column != null) { - int compare = Bytes.compareTo(column.getBuffer(), column.getOffset(), - column.getLength(), bytes, offset, length); + int compare; + if (columnCell.hasArray()) { + compare = Bytes.compareTo(column.getBuffer(), column.getOffset(), column.getLength(), + columnCell.getQualifierArray(), offset, length); + } else { + compare = ByteBufferUtils.compareTo(column.getBuffer(), column.getOffset(), + column.getLength(), columnCell.getQualifierBuffer(), offset, length); + } resetTS(); if (compare <= 0) { ++this.index; @@ -239,9 +253,10 @@ public class ExplicitColumnTracker implements ColumnTracker { } } - public MatchCode getNextRowOrNextColumn(byte[] bytes, int offset, + @Override + public MatchCode getNextRowOrNextColumn(Cell columnCell, int offset, int qualLength) { - doneWithColumn(bytes, offset,qualLength); + doneWithColumn(columnCell, offset, qualLength); if (getColumnHint() == null) { return MatchCode.SEEK_NEXT_ROW; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java index 4d22c0e..7bbff1f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java @@ -30,7 +30,7 @@ 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.util.Bytes; +import org.apache.hadoop.hbase.util.ByteBufferUtils; /** * State and utility processing {@link HRegion#getClosestRowBefore(byte[], byte[])}. @@ -143,9 +143,9 @@ class GetClosestRowBeforeTracker { continue; } // Check column - int ret = Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), + int ret = ByteBufferUtils.compareTo(kv.getQualifierBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), - d.getQualifierArray(), d.getQualifierOffset(), d.getQualifierLength()); + d.getQualifierBuffer(), d.getQualifierOffset(), d.getQualifierLength()); if (ret <= -1) { // This delete is for an earlier column. continue; @@ -238,8 +238,8 @@ class GetClosestRowBeforeTracker { if (!metaregion) return true; // Compare start of keys row. Compare including delimiter. Saves having // to calculate where tablename ends in the candidate kv. - return Bytes.compareTo(this.targetkey.getRowArray(), this.rowoffset, + return ByteBufferUtils.compareTo(this.targetkey.getRowBuffer(), this.rowoffset, this.tablenamePlusDelimiterLength, - kv.getRowArray(), kv.getRowOffset(), this.tablenamePlusDelimiterLength) == 0; + kv.getRowBuffer(), kv.getRowOffset(), this.tablenamePlusDelimiterLength) == 0; } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 3b1f267..f7c4314 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.io.UnsupportedEncodingException; import java.lang.reflect.Constructor; +import java.nio.ByteBuffer; import java.text.ParseException; import java.util.AbstractList; import java.util.ArrayList; @@ -69,6 +70,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -88,7 +90,9 @@ import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -152,6 +156,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.ClassSize; @@ -3411,8 +3416,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // 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; + if (kv.hasArray()) { + compareResult = comparator.compareTo(kv.getValueArray(), kv.getValueOffset(), + kv.getValueLength()); + } else { + compareResult = comparator.compareTo(kv.getValueBuffer(), kv.getValueOffset(), + kv.getValueLength()); + } switch (compareOp) { case LESS: matches = compareResult < 0; @@ -3608,8 +3619,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // for (int i = 0; i < listSize; i++) { Cell cell = cells.get(i); List newTags = new ArrayList(); - Iterator tagIterator = CellUtil.tagsIterator(cell.getTagsArray(), - cell.getTagsOffset(), cell.getTagsLength()); + Iterator tagIterator = CellUtil.tagsIterator(cell); // Carry forward existing tags @@ -3626,17 +3636,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // above may change when there are more tag based features in core. if (m.getTTL() != Long.MAX_VALUE) { // Add a cell TTL tag - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL()))); + newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL()))); } // Rewrite the cell with the updated set of tags - - cells.set(i, new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), - cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), - cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), - cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()), - cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(), - newTags)); + cells.set(i,new TagRewriteCell(cell, TagUtil.fromList(newTags))); } } } @@ -5562,9 +5566,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // private NextState populateFromJoinedHeap(List results, int limit, long resultSize) throws IOException { assert joinedContinuationRow != null; + // TODO : Handle BB and byte[] for joinedContinuationRow NextState state = populateResult(results, this.joinedHeap, limit, resultSize, - joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(), + joinedContinuationRow.getRowBuffer(), joinedContinuationRow.getRowOffset(), joinedContinuationRow.getRowLength()); if (state != null && !state.batchLimitReached() && !state.sizeLimitReached()) { // We are done with this row, reset the continuation. @@ -5589,7 +5594,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @return state of last call to {@link KeyValueHeap#next()} */ private NextState populateResult(List results, KeyValueHeap heap, int batchLimit, - long remainingResultSize, byte[] currentRow, int offset, short length) throws IOException { + long remainingResultSize, ByteBuffer currentRow, int offset, short length) throws IOException { Cell nextKv; boolean moreCellsInRow = false; long accumulatedResultSize = 0; @@ -5632,7 +5637,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // * @param length * @return true When there are more cells in the row to be read */ - private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset, +/* private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset, + short length) { + return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length); + }*/ + + private boolean moreCellsInRow(final Cell nextKv, ByteBuffer currentRow, int offset, short length) { return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length); } @@ -5704,11 +5714,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // Let's see what we have in the storeHeap. Cell current = this.storeHeap.peek(); - byte[] currentRow = null; + ByteBuffer currentRow = null; int offset = 0; short length = 0; if (current != null) { - currentRow = current.getRowArray(); + currentRow = current.getRowBuffer(); offset = current.getRowOffset(); length = current.getRowLength(); } @@ -5781,7 +5791,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } Cell nextKv = this.storeHeap.peek(); stopRow = nextKv == null || - isStopRow(nextKv.getRowArray(), nextKv.getRowOffset(), nextKv.getRowLength()); + isStopRow(nextKv.getRowBuffer(), nextKv.getRowOffset(), nextKv.getRowLength()); // save that the row was empty before filters applied to it. final boolean isEmptyRow = results.isEmpty(); @@ -5813,7 +5823,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // boolean mayHaveData = (nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv, currentRow, offset, length)) || (this.joinedHeap.requestSeek( - KeyValueUtil.createFirstOnRow(currentRow, offset, length), true, true) + CellUtil.createFirstOnRow(current), true, true) && joinedHeap.peek() != null && CellUtil.matchingRow(joinedHeap.peek(), currentRow, offset, length)); if (mayHaveData) { @@ -5877,12 +5887,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // && filter.filterRow(); } - private boolean filterRowKey(byte[] row, int offset, short length) throws IOException { + private boolean filterRowKey(ByteBuffer row, int offset, short length) throws IOException { return filter != null && filter.filterRowKey(row, offset, length); } - - protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException { + + protected boolean nextRow(ByteBuffer currentRow, int offset, short length) throws IOException { assert this.joinedContinuationRow == null: "Trying to go to next row during joinedHeap read."; Cell next; while ((next = this.storeHeap.peek()) != null && @@ -5896,7 +5906,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // .postScannerFilterRow(this, currentRow, offset, length); } - protected boolean isStopRow(byte[] currentRow, int offset, short length) { + protected boolean isStopRow(ByteBuffer currentRow, int offset, short length) { return currentRow == null || (stopRow != null && comparator.compareRows(stopRow, 0, stopRow.length, @@ -6851,7 +6861,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // long now = EnvironmentEdgeManager.currentTime(); // Process each family for (Map.Entry> family : append.getFamilyCellMap().entrySet()) { - Store store = stores.get(family.getKey()); List kvs = new ArrayList(family.getValue().size()); @@ -6905,37 +6914,61 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (append.getTTL() != Long.MAX_VALUE) { // Add the new TTL tag - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append.getTTL()))); + newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append + .getTTL()))); } // Rebuild tags - byte[] tagBytes = Tag.fromList(newTags); + byte[] tagBytes = TagUtil.fromList(newTags); // allocate an empty cell once newCell = new KeyValue(row.length, cell.getFamilyLength(), cell.getQualifierLength(), ts, KeyValue.Type.Put, oldCell.getValueLength() + cell.getValueLength(), tagBytes.length); - // copy in row, family, and qualifier - System.arraycopy(cell.getRowArray(), cell.getRowOffset(), - newCell.getRowArray(), newCell.getRowOffset(), cell.getRowLength()); - System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), - newCell.getFamilyArray(), newCell.getFamilyOffset(), - cell.getFamilyLength()); - System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), - newCell.getQualifierArray(), newCell.getQualifierOffset(), - cell.getQualifierLength()); // copy in the value - System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(), - newCell.getValueArray(), newCell.getValueOffset(), - oldCell.getValueLength()); - System.arraycopy(cell.getValueArray(), cell.getValueOffset(), - newCell.getValueArray(), - newCell.getValueOffset() + oldCell.getValueLength(), - cell.getValueLength()); - // Copy in tag data - System.arraycopy(tagBytes, 0, newCell.getTagsArray(), newCell.getTagsOffset(), - tagBytes.length); + /*System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(), + newCell.getValueArray(), newCell.getValueOffset(), + oldCell.getValueLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getValueArray(), + oldCell.getValueBuffer(), oldCell.getValueOffset(), newCell.getValueOffset(), + oldCell.getValueLength()); + /* System.arraycopy(cell.getValueArray(), cell.getValueOffset(), + newCell.getValueArray(), + newCell.getValueOffset() + oldCell.getValueLength(), + cell.getValueLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getValueArray(), + cell.getValueBuffer(), cell.getValueOffset(), newCell.getValueOffset() + oldCell.getValueLength(), + cell.getValueLength()); + // copy in the tags + /* System.arraycopy(oldCell.getTagsBuffer(), oldCell.getTagsOffset(), + newCell.getTagsBuffer(), newCell.getTagsOffset(), oldCell.getTagsLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getTagsArray(), + oldCell.getTagsBuffer(), oldCell.getTagsOffset(), newCell.getTagsOffset(), + oldCell.getTagsLength()); + /* System.arraycopy(cell.getTagsBuffer(), cell.getTagsOffset(), newCell.getTagsBuffer(), + newCell.getTagsOffset() + oldCell.getTagsLength(), cell.getTagsLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getTagsArray(), + cell.getTagsBuffer(), cell.getTagsOffset(), newCell.getTagsOffset() + oldCell.getTagsLength(), + cell.getTagsLength()); + // copy in row, family, and qualifier + /*System.arraycopy(cell.getRowBuffer(), cell.getRowOffset(), + newCell.getRowBuffer(), newCell.getRowOffset(), cell.getRowLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getRowArray(), + cell.getRowBuffer(), cell.getRowOffset(), newCell.getRowOffset(), + cell.getRowLength()); +/* System.arraycopy(cell.getFamilyBuffer(), cell.getFamilyOffset(), + newCell.getFamilyBuffer(), newCell.getFamilyOffset(), + cell.getFamilyLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getFamilyArray(), + cell.getFamilyBuffer(), cell.getFamilyOffset(), newCell.getFamilyOffset(), + cell.getFamilyLength()); + /*System.arraycopy(cell.getQualifierBuffer(), cell.getQualifierOffset(), + newCell.getQualifierBuffer(), newCell.getQualifierOffset(), + cell.getQualifierLength());*/ + ByteBufferUtils.copyFromBufferToByteArray(newCell.getQualifierArray(), + cell.getQualifierBuffer(), cell.getQualifierOffset(), newCell.getQualifierOffset(), + cell.getQualifierLength()); idx++; } else { // Append's KeyValue.Type==Put and ts==HConstants.LATEST_TIMESTAMP @@ -6945,7 +6978,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // if (append.getTTL() != Long.MAX_VALUE) { List newTags = new ArrayList(1); - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append.getTTL()))); + newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append + .getTTL()))); // Add the new TTL tag newCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), @@ -7160,7 +7194,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // c = results.get(idx); ts = Math.max(now, c.getTimestamp()); if(c.getValueLength() == Bytes.SIZEOF_LONG) { - amount += Bytes.toLong(c.getValueArray(), c.getValueOffset(), Bytes.SIZEOF_LONG); + amount += ByteBufferUtils.getLong(c.getValueBuffer(), c.getValueOffset()); } else { // throw DoNotRetryIOException instead of IllegalArgumentException throw new org.apache.hadoop.hbase.DoNotRetryIOException( @@ -7184,9 +7218,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // // Add the TTL tag if the mutation carried one if (increment.getTTL() != Long.MAX_VALUE) { - newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(increment.getTTL()))); + newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(increment + .getTTL()))); } - Cell newKV = new KeyValue(row, 0, row.length, family.getKey(), 0, family.getKey().length, q, 0, q.length, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 686df49..ad68635 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputCont import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.ClassSize; @@ -1764,8 +1765,7 @@ public class HStore implements Store { // Look for a TTL tag first. Use it instead of the family setting if // found. If a cell has multiple TTLs, resolve the conflict by using the // first tag encountered. - Iterator i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator i = CellUtil.tagsIterator(cell); while (i.hasNext()) { Tag t = i.next(); if (TagType.TTL_TAG_TYPE == t.getType()) { @@ -1773,7 +1773,12 @@ public class HStore implements Store { // to convert long ts = cell.getTimestamp(); assert t.getTagLength() == Bytes.SIZEOF_LONG; - long ttl = Bytes.toLong(t.getBuffer(), t.getTagOffset(), t.getTagLength()); + long ttl = 0; + if (cell.hasArray()) { + ttl = Bytes.toLong(t.getBytes(), t.getTagOffset(), t.getTagLength()); + } else { + ttl = ByteBufferUtils.getAsLong(t.getBuffer(), t.getTagOffset(), t.getTagLength()); + } if (ts + ttl < now) { return true; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java index 5c3c1e7..9cff5de 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLAB.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.ByteRange; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 3ce1b57..02e83e7 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -1760,6 +1760,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } } + // TODO add cells to PayloadCaryyingRpcController if (existence != null){ ClientProtos.Result pbr = ProtobufUtil.toResult(existence, region.getRegionInfo().getReplicaId() != 0); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index d10141c..62cd706 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -1381,6 +1382,28 @@ public class RegionCoprocessorHost } /** + * This will be called by the scan flow when the current scanned row is being filtered out by the + * filter. + * @param s the scanner + * @param currentRow The current rowkey which got filtered out + * @param offset offset to rowkey + * @param length length of rowkey + * @return whether more rows are available for the scanner or not + * @throws IOException + */ + public boolean postScannerFilterRow(final InternalScanner s, final ByteBuffer currentRow, + final int offset, final short length) throws IOException { + return execOperationWithResult(true, + coprocessors.isEmpty() ? null : new RegionOperationWithResult() { + @Override + public void call(RegionObserver oserver, ObserverContext ctx) + throws IOException { + setResult(oserver.postScannerFilterRow(ctx, s, currentRow, offset,length, getResult())); + } + }); + } + + /** * @param s the scanner * @return true if default behavior should be bypassed, false otherwise * @exception IOException Exception diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java index c7ce180..7463339 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java @@ -77,8 +77,8 @@ public class ReversedKeyValueHeap extends KeyValueHeap { KeyValueScanner scanner; while ((scanner = heap.poll()) != null) { Cell topKey = scanner.peek(); - if (comparator.getComparator().compareRows(topKey.getRowArray(), - topKey.getRowOffset(), topKey.getRowLength(), seekKey.getRowArray(), + if (comparator.getComparator().compareRows(topKey.getRowBuffer(), + topKey.getRowOffset(), topKey.getRowLength(), seekKey.getRowBuffer(), seekKey.getRowOffset(), seekKey.getRowLength()) < 0) { // Row of Top KeyValue is before Seek row. heap.add(scanner); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java index feda699..5ceffb9 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java @@ -19,10 +19,10 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; @@ -55,7 +55,6 @@ class ReversedRegionScannerImpl extends RegionScannerImpl { } } - @Override protected boolean isStopRow(byte[] currentRow, int offset, short length) { return currentRow == null || (super.stopRow != null && region.getComparator().compareRows( @@ -63,19 +62,9 @@ class ReversedRegionScannerImpl extends RegionScannerImpl { } @Override - protected boolean nextRow(byte[] currentRow, int offset, short length) - throws IOException { - assert super.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read."; - byte row[] = new byte[length]; - System.arraycopy(currentRow, offset, row, 0, length); - this.storeHeap.seekToPreviousRow(KeyValueUtil.createFirstOnRow(row)); - resetFilters(); - // Calling the hook in CP which allows it to do a fast forward - if (this.region.getCoprocessorHost() != null) { - return this.region.getCoprocessorHost().postScannerFilterRow(this, - currentRow, offset, length); - } - return true; + protected boolean isStopRow(ByteBuffer currentRow, int offset, short length) { + return currentRow == null + || (super.stopRow != null && region.getComparator().compareRows( + stopRow, 0, stopRow.length, currentRow, offset, length) >= super.isScan); } - } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java index a5c17fb..80bc5ee 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanDeleteTracker.java @@ -19,12 +19,14 @@ package org.apache.hadoop.hbase.regionserver; +import java.nio.ByteBuffer; import java.util.SortedSet; import java.util.TreeSet; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; /** @@ -48,7 +50,7 @@ public class ScanDeleteTracker implements DeleteTracker { protected boolean hasFamilyStamp = false; protected long familyStamp = 0L; protected SortedSet familyVersionStamps = new TreeSet(); - protected byte [] deleteBuffer = null; + protected ByteBuffer deleteBuffer = null; protected int deleteOffset = 0; protected int deleteLength = 0; protected byte deleteType = 0; @@ -86,13 +88,14 @@ public class ScanDeleteTracker implements DeleteTracker { if (deleteBuffer != null && type < deleteType) { // same column, so ignore less specific delete - if (Bytes.equals(deleteBuffer, deleteOffset, deleteLength, - cell.getQualifierArray(), qualifierOffset, qualifierLength)){ + // easy to go with buffers when Cell has buffers + if (ByteBufferUtils.equals(deleteBuffer, deleteOffset, deleteLength, + cell.getQualifierBuffer(), qualifierOffset, qualifierLength)){ return; } } // new column, or more general delete type - deleteBuffer = cell.getQualifierArray(); + deleteBuffer = cell.getQualifierBuffer(); deleteOffset = qualifierOffset; deleteLength = qualifierLength; deleteType = type; @@ -122,8 +125,8 @@ public class ScanDeleteTracker implements DeleteTracker { } if (deleteBuffer != null) { - int ret = Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength, - cell.getQualifierArray(), qualifierOffset, qualifierLength); + int ret = ByteBufferUtils.compareTo(deleteBuffer, deleteOffset, deleteLength, + cell.getQualifierBuffer(), qualifierOffset, qualifierLength); if (ret == 0) { if (deleteType == KeyValue.Type.DeleteColumn.getCode()) { @@ -144,7 +147,7 @@ public class ScanDeleteTracker implements DeleteTracker { deleteBuffer = null; } else { throw new IllegalStateException("isDelete failed: deleteBuffer=" - + Bytes.toStringBinary(deleteBuffer, deleteOffset, deleteLength) + + ByteBufferUtils.toStringBinary(deleteBuffer, deleteOffset, deleteLength) + ", qualifier=" + Bytes.toStringBinary(cell.getQualifierArray(), qualifierOffset, qualifierLength) + ", timestamp=" + timestamp + ", comparison result: " + ret); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java index 032b4ce..e24cf40 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java @@ -20,12 +20,14 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.NavigableSet; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.FirstOnColumnCell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; @@ -35,6 +37,7 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter.ReturnCode; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.regionserver.DeleteTracker.DeleteResult; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -88,12 +91,10 @@ public class ScanQueryMatcher { /** Row comparator for the region this query is for */ private final KeyValue.KVComparator rowComparator; - /* row is not private for tests */ - /** Row the query is on */ - byte [] row; + Cell rowCell; int rowOffset; short rowLength; - + /** * Oldest put in any of the involved store files * Used to decide whether it is ok to delete @@ -165,8 +166,9 @@ public class ScanQueryMatcher { this.regionCoprocessorHost = regionCoprocessorHost; this.deletes = instantiateDeleteTracker(); this.stopRow = scan.getStopRow(); - this.startKey = KeyValueUtil.createFirstDeleteFamilyOnRow(scan.getStartRow(), - scanInfo.getFamily()); + this.startKey = KeyValueUtil.createFirstDeleteFamilyOnRow( + scan.getStartRow(), + scanInfo.getFamily() == null ? new byte[0] : scanInfo.getFamily()); this.filter = scan.getFilter(); this.earliestPutTs = earliestPutTs; this.oldestUnexpiredTS = oldestUnexpiredTS; @@ -278,8 +280,9 @@ public class ScanQueryMatcher { if (filter != null && filter.filterAllRemaining()) { return MatchCode.DONE_SCAN; } - int ret = this.rowComparator.compareRows(row, this.rowOffset, this.rowLength, - cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); + // TODO + int ret = this.rowComparator.compareRows(rowCell.getRowBuffer(), this.rowOffset, this.rowLength, + cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength()); if (!this.isReversed) { if (ret <= -1) { return MatchCode.DONE; @@ -298,8 +301,7 @@ public class ScanQueryMatcher { } // optimize case. - if (this.stickyNextRow) - return MatchCode.SEEK_NEXT_ROW; + if (this.stickyNextRow) return MatchCode.SEEK_NEXT_ROW; if (this.columns.done()) { stickyNextRow = true; @@ -312,8 +314,7 @@ public class ScanQueryMatcher { long timestamp = cell.getTimestamp(); // check for early out based on timestamp alone if (columns.isDone(timestamp)) { - return columns.getNextRowOrNextColumn(cell.getQualifierArray(), qualifierOffset, - qualifierLength); + return columns.getNextRowOrNextColumn(cell, qualifierOffset, qualifierLength); } // check if the cell is expired by cell TTL if (HStore.isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) { @@ -334,8 +335,8 @@ public class ScanQueryMatcher { * they affect */ byte typeByte = cell.getTypeByte(); - long mvccVersion = cell.getMvccVersion(); - if (CellUtil.isDelete(cell)) { + long mvccVersion = cell.getSequenceId(); + if (CellUtil.isDelete(typeByte)) { if (keepDeletedCells == KeepDeletedCells.FALSE || (keepDeletedCells == KeepDeletedCells.TTL && timestamp < ttl)) { // first ignore delete markers if the scanner can do so, and the @@ -372,8 +373,7 @@ public class ScanQueryMatcher { if (timestamp < earliestPutTs) { // keeping delete rows, but there are no puts older than // this delete in the store files. - return columns.getNextRowOrNextColumn(cell.getQualifierArray(), - qualifierOffset, qualifierLength); + return columns.getNextRowOrNextColumn(cell, qualifierOffset, qualifierLength); } // else: fall through and do version counting on the // delete markers @@ -387,8 +387,7 @@ public class ScanQueryMatcher { switch (deleteResult) { case FAMILY_DELETED: case COLUMN_DELETED: - return columns.getNextRowOrNextColumn(cell.getQualifierArray(), - qualifierOffset, qualifierLength); + return columns.getNextRowOrNextColumn(cell, qualifierOffset, qualifierLength); case VERSION_DELETED: case FAMILY_VERSION_DELETED: return MatchCode.SKIP; @@ -403,13 +402,13 @@ public class ScanQueryMatcher { if (timestampComparison >= 1) { return MatchCode.SKIP; } else if (timestampComparison <= -1) { - return columns.getNextRowOrNextColumn(cell.getQualifierArray(), qualifierOffset, - qualifierLength); + return columns.getNextRowOrNextColumn(cell, qualifierOffset, qualifierLength); } // STEP 1: Check if the column is part of the requested columns - MatchCode colChecker = columns.checkColumn(cell.getQualifierArray(), - qualifierOffset, qualifierLength, typeByte); + MatchCode colChecker = columns.checkColumn(cell, qualifierOffset, qualifierLength, + typeByte); + if (colChecker == MatchCode.INCLUDE) { ReturnCode filterResponse = ReturnCode.SKIP; // STEP 2: Yes, the column is part of the requested columns. Check if filter is present @@ -420,8 +419,7 @@ public class ScanQueryMatcher { case SKIP: return MatchCode.SKIP; case NEXT_COL: - return columns.getNextRowOrNextColumn(cell.getQualifierArray(), - qualifierOffset, qualifierLength); + return columns.getNextRowOrNextColumn(cell, qualifierOffset, qualifierLength); case NEXT_ROW: stickyNextRow = true; return MatchCode.SEEK_NEXT_ROW; @@ -451,10 +449,9 @@ public class ScanQueryMatcher { * In all the above scenarios, we return the column checker return value except for * FilterResponse (INCLUDE_AND_SEEK_NEXT_COL) and ColumnChecker(INCLUDE) */ - colChecker = - columns.checkVersions(cell.getQualifierArray(), qualifierOffset, - qualifierLength, timestamp, typeByte, - mvccVersion > maxReadPointToTrackVersions); + // Directly use buffer here + colChecker = columns.checkVersions(cell, qualifierOffset, qualifierLength, timestamp, + typeByte, mvccVersion > maxReadPointToTrackVersions); //Optimize with stickyNextRow stickyNextRow = colChecker == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW ? true : stickyNextRow; return (filterResponse == ReturnCode.INCLUDE_AND_NEXT_COL && @@ -469,15 +466,14 @@ public class ScanQueryMatcher { /** Handle partial-drop-deletes. As we match keys in order, when we have a range from which * we can drop deletes, we can set retainDeletesInOutput to false for the duration of this * range only, and maintain consistency. */ - private void checkPartialDropDeleteRange(byte [] row, int offset, short length) { + private void checkPartialDropDeleteRange(Cell c) { // If partial-drop-deletes are used, initially, dropDeletesFromRow and dropDeletesToRow // are both set, and the matcher is set to retain deletes. We assume ordered keys. When // dropDeletesFromRow is leq current kv, we start dropping deletes and reset // dropDeletesFromRow; thus the 2nd "if" starts to apply. if ((dropDeletesFromRow != null) && ((dropDeletesFromRow == HConstants.EMPTY_START_ROW) - || (Bytes.compareTo(row, offset, length, - dropDeletesFromRow, 0, dropDeletesFromRow.length) >= 0))) { + || (compareRow(c, dropDeletesFromRow, 0, dropDeletesFromRow.length) >= 0))) { retainDeletesInOutput = false; dropDeletesFromRow = null; } @@ -486,25 +482,32 @@ public class ScanQueryMatcher { // and reset dropDeletesToRow so that we don't do any more compares. if ((dropDeletesFromRow == null) && (dropDeletesToRow != null) && (dropDeletesToRow != HConstants.EMPTY_END_ROW) - && (Bytes.compareTo(row, offset, length, - dropDeletesToRow, 0, dropDeletesToRow.length) >= 0)) { + && (compareRow(c, dropDeletesToRow, 0, dropDeletesToRow.length) >= 0)) { retainDeletesInOutput = true; dropDeletesToRow = null; } } + private int compareRow(Cell c, byte[] b, int offset, int length) { + if (c.hasArray()) { + return Bytes + .compareTo(c.getRowArray(), c.getRowOffset(), c.getRowLength(), b, offset, length); + } + return ByteBufferUtils.compareTo(c.getRowBuffer(), c.getRowOffset(), c.getRowLength(), b, + offset, length); + } + public boolean moreRowsMayExistAfter(Cell kv) { if (this.isReversed) { - if (rowComparator.compareRows(kv.getRowArray(), kv.getRowOffset(), - kv.getRowLength(), stopRow, 0, stopRow.length) <= 0) { + if (rowComparator.compareRows(kv, stopRow, 0, stopRow.length) <= 0) { return false; } else { return true; } } + if (!Bytes.equals(stopRow , HConstants.EMPTY_END_ROW) && - rowComparator.compareRows(kv.getRowArray(),kv.getRowOffset(), - kv.getRowLength(), stopRow, 0, stopRow.length) >= 0) { + rowComparator.compareRows(kv, stopRow, 0, stopRow.length) >= 0) { // KV >= STOPROW // then NO there is nothing left. return false; @@ -513,15 +516,22 @@ public class ScanQueryMatcher { } } + // Only used by tests + void setRow(byte[] row, int offset, short length) { + setRow(KeyValueUtil.createFirstOnRow(row, offset, length), offset, length); + } + /** * Set current row * @param row + * @param rowOffset + * @param rowLength */ - public void setRow(byte [] row, int offset, short length) { - checkPartialDropDeleteRange(row, offset, length); - this.row = row; - this.rowOffset = offset; - this.rowLength = length; + public void setRow(Cell c, int rowOffset, short rowLength) { + checkPartialDropDeleteRange(c); + this.rowCell = c; + this.rowOffset = rowOffset; + this.rowLength = rowLength; reset(); } @@ -559,23 +569,22 @@ public class ScanQueryMatcher { public Cell getKeyForNextColumn(Cell kv) { ColumnCount nextColumn = columns.getColumnHint(); if (nextColumn == null) { - return KeyValueUtil.createLastOnRow( - kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), - kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), - kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()); + return CellUtil.createLastOnColumn(kv); } else { - return KeyValueUtil.createFirstOnRow( - kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), - kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), - nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength()); + if (kv.hasArray()) { + return KeyValueUtil.createFirstOnRow(kv.getRowArray(), kv.getRowOffset(), + kv.getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), + nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength()); + } else { + return new FirstOnColumnCell(kv.getRowBuffer(), kv.getRowOffset(), kv.getRowLength(), + kv.getFamilyBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), + ByteBuffer.wrap(nextColumn.getBuffer()), nextColumn.getOffset(), nextColumn.getLength()); + } } } public Cell getKeyForNextRow(Cell kv) { - return KeyValueUtil.createLastOnRow( - kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), - null, 0, 0, - null, 0, 0); + return CellUtil.createLastOnRow(kv); } /** @@ -584,11 +593,15 @@ public class ScanQueryMatcher { * @return result of the compare between the indexed key and the key portion of the passed cell */ public int compareKeyForNextRow(Cell nextIndexed, Cell kv) { - return rowComparator.compareKey(nextIndexed, - kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), - null, 0, 0, - null, 0, 0, - HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); + if (kv.hasArray()) { + return rowComparator.compareKey(nextIndexed, kv.getRowArray(), kv.getRowOffset(), + kv.getRowLength(), null, 0, 0, null, 0, 0, HConstants.OLDEST_TIMESTAMP, + Type.Minimum.getCode()); + } else { + return rowComparator.compareKey(nextIndexed, kv.getRowBuffer(), kv.getRowOffset(), + kv.getRowLength(), (ByteBuffer)null, 0, 0, (ByteBuffer)null, 0, 0, HConstants.OLDEST_TIMESTAMP, + Type.Minimum.getCode()); + } } /** @@ -598,27 +611,48 @@ public class ScanQueryMatcher { */ public int compareKeyForNextColumn(Cell nextIndexed, Cell kv) { ColumnCount nextColumn = columns.getColumnHint(); - if (nextColumn == null) { - return rowComparator.compareKey(nextIndexed, - kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), - kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), - kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), - HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); + if (kv.hasArray()) { + if (nextColumn == null) { + // TODO : better to create FirstOnXXXCell and LastOnXXXCell here rather + // than this + return rowComparator.compareKey(nextIndexed, kv.getRowArray(), kv.getRowOffset(), + kv.getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), + kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), + HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); + } else { + return rowComparator.compareKey(nextIndexed, kv.getRowArray(), kv.getRowOffset(), + kv.getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), + nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength(), + HConstants.LATEST_TIMESTAMP, Type.Maximum.getCode()); + } } else { - return rowComparator.compareKey(nextIndexed, - kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), - kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), - nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength(), - HConstants.LATEST_TIMESTAMP, Type.Maximum.getCode()); + if (nextColumn == null) { + // TODO : better to create FirstOnXXXCell and LastOnXXXCell here rather + // than this + return rowComparator.compareKey(nextIndexed, kv.getRowBuffer(), kv.getRowOffset(), + kv.getRowLength(), kv.getFamilyBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), + kv.getQualifierBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(), + HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode()); + } else { + // TODO : am going to do a wrap here. This may be costly to avoid that create one more version of compareKey + return rowComparator.compareKey(nextIndexed, kv.getRowBuffer(), kv.getRowOffset(), + kv.getRowLength(), kv.getFamilyBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(), + nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength(), + HConstants.LATEST_TIMESTAMP, Type.Maximum.getCode()); + } } } - //Used only for testing purposes + // Used only for testing purposes static MatchCode checkColumn(ColumnTracker columnTracker, byte[] bytes, int offset, int length, long ttl, byte type, boolean ignoreCount) throws IOException { - MatchCode matchCode = columnTracker.checkColumn(bytes, offset, length, type); + Cell c = KeyValueUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY, 0, 0, + HConstants.EMPTY_BYTE_ARRAY, 0, 0, bytes, offset, length); + MatchCode matchCode = columnTracker.checkColumn(c, c.getQualifierOffset(), + c.getQualifierLength(), type); if (matchCode == MatchCode.INCLUDE) { - return columnTracker.checkVersions(bytes, offset, length, ttl, type, ignoreCount); + return columnTracker.checkVersions(c, c.getQualifierOffset(), c.getQualifierLength(), ttl, + type, ignoreCount); } return matchCode; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java index 85b36fb..232581e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanWildcardColumnTracker.java @@ -21,20 +21,22 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import org.apache.hadoop.hbase.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.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode; -import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ByteBufferUtils; /** * Keeps track of the columns for a scan if they are not explicitly specified */ @InterfaceAudience.Private public class ScanWildcardColumnTracker implements ColumnTracker { - private byte [] columnBuffer = null; - private int columnOffset = 0; - private int columnLength = 0; + private Cell curCell = null; + private int columnOffset = 0; // Offset to qualifier in the curCell + private int columnLength = 0; // length of qualifier in the curCell private int currentCount = 0; private int maxVersions; private int minVersions; @@ -64,7 +66,7 @@ public class ScanWildcardColumnTracker implements ColumnTracker { * This receives puts *and* deletes. */ @Override - public MatchCode checkColumn(byte[] bytes, int offset, int length, byte type) + public MatchCode checkColumn(Cell columnCell, int offset, int length, byte type) throws IOException { return MatchCode.INCLUDE; } @@ -75,18 +77,18 @@ public class ScanWildcardColumnTracker implements ColumnTracker { * take the version of the previous put (so eventually all but the last can be reclaimed). */ @Override - public ScanQueryMatcher.MatchCode checkVersions(byte[] bytes, int offset, int length, + public ScanQueryMatcher.MatchCode checkVersions(Cell columnCell, int offset, int length, long timestamp, byte type, boolean ignoreCount) throws IOException { - if (columnBuffer == null) { + if (curCell == null) { // first iteration. - resetBuffer(bytes, offset, length); + resetColumn(columnCell, offset, length); if (ignoreCount) return ScanQueryMatcher.MatchCode.INCLUDE; // do not count a delete marker as another version return checkVersion(type, timestamp); } - int cmp = Bytes.compareTo(bytes, offset, length, - columnBuffer, columnOffset, columnLength); + int cmp = CellComparator.compareQualifiers(columnCell, offset, length, curCell, columnOffset, + columnLength); if (cmp == 0) { if (ignoreCount) return ScanQueryMatcher.MatchCode.INCLUDE; @@ -102,7 +104,7 @@ public class ScanWildcardColumnTracker implements ColumnTracker { // new col > old col if (cmp > 0) { // switched columns, lets do something.x - resetBuffer(bytes, offset, length); + resetColumn(columnCell, offset, length); if (ignoreCount) return ScanQueryMatcher.MatchCode.INCLUDE; return checkVersion(type, timestamp); } @@ -114,11 +116,11 @@ public class ScanWildcardColumnTracker implements ColumnTracker { throw new IOException( "ScanWildcardColumnTracker.checkColumn ran into a column actually " + "smaller than the previous column: " + - Bytes.toStringBinary(bytes, offset, length)); + ByteBufferUtils.toStringBinary(columnCell.getQualifierBuffer(), offset, length)); } - private void resetBuffer(byte[] bytes, int offset, int length) { - columnBuffer = bytes; + private void resetColumn(Cell cell, int offset, int length) { + curCell = cell; columnOffset = offset; columnLength = length; currentCount = 0; @@ -152,7 +154,7 @@ public class ScanWildcardColumnTracker implements ColumnTracker { @Override public void reset() { - columnBuffer = null; + curCell = null; resetTSAndType(); } @@ -181,6 +183,7 @@ public class ScanWildcardColumnTracker implements ColumnTracker { * * @return The column count. */ + @Override public ColumnCount getColumnHint() { return null; } @@ -194,7 +197,8 @@ public class ScanWildcardColumnTracker implements ColumnTracker { return false; } - public MatchCode getNextRowOrNextColumn(byte[] bytes, int offset, + @Override + public MatchCode getNextRowOrNextColumn(Cell columnCell, int offset, int qualLength) { return MatchCode.SEEK_NEXT_COL; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index c1a6b76..5416435 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.HFileWriterV2; @@ -56,7 +57,10 @@ import org.apache.hadoop.hbase.regionserver.compactions.Compactor; import org.apache.hadoop.hbase.util.BloomFilter; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.WritableUtils; @@ -707,6 +711,8 @@ public class StoreFile { private final BloomFilterWriter deleteFamilyBloomFilterWriter; private final BloomType bloomType; private byte[] lastBloomKey; + // TODO : Better name + private ByteBuffer lastBloomKey_BR; private int lastBloomKeyOffset, lastBloomKeyLen; private KVComparator kvComparator; private Cell lastCell = null; @@ -861,44 +867,82 @@ public class StoreFile { * 1. Row = Row * 2. RowCol = Row + Qualifier */ - byte[] bloomKey; - int bloomKeyOffset, bloomKeyLen; - - switch (bloomType) { - case ROW: - bloomKey = cell.getRowArray(); - bloomKeyOffset = cell.getRowOffset(); - bloomKeyLen = cell.getRowLength(); - break; - case ROWCOL: - // merge(row, qualifier) - // TODO: could save one buffer copy in case of compound Bloom - // filters when this involves creating a KeyValue - bloomKey = generalBloomFilterWriter.createBloomKey(cell.getRowArray(), - cell.getRowOffset(), cell.getRowLength(), cell.getQualifierArray(), - cell.getQualifierOffset(), cell.getQualifierLength()); - bloomKeyOffset = 0; - bloomKeyLen = bloomKey.length; - break; - default: - throw new IOException("Invalid Bloom filter type: " + bloomType + - " (ROW or ROWCOL expected)"); - } - generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); - if (lastBloomKey != null - && generalBloomFilterWriter.getComparator().compareFlatKey(bloomKey, - bloomKeyOffset, bloomKeyLen, lastBloomKey, - lastBloomKeyOffset, lastBloomKeyLen) <= 0) { - throw new IOException("Non-increasing Bloom keys: " - + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) - + " after " - + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, - lastBloomKeyLen)); + // TODO : This needs major refactoring + if (cell.hasArray()) { + byte[] bloomKey; + int bloomKeyOffset, bloomKeyLen; + + switch (bloomType) { + case ROW: + bloomKey = cell.getRowArray(); + bloomKeyOffset = cell.getRowOffset(); + bloomKeyLen = cell.getRowLength(); + break; + case ROWCOL: + // merge(row, qualifier) + // TODO: could save one buffer copy in case of compound Bloom + // filters when this involves creating a KeyValue + bloomKey = generalBloomFilterWriter.createBloomKey(cell.getRowArray(), + cell.getRowOffset(), cell.getRowLength(), cell.getQualifierArray(), + cell.getQualifierOffset(), cell.getQualifierLength()); + bloomKeyOffset = 0; + bloomKeyLen = bloomKey.length; + break; + default: + throw new IOException("Invalid Bloom filter type: " + bloomType + + " (ROW or ROWCOL expected)"); + } + generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); + if (lastBloomKey != null + && generalBloomFilterWriter.getComparator().compareFlatKey(bloomKey, + bloomKeyOffset, bloomKeyLen, lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen) <= 0) { + throw new IOException("Non-increasing Bloom keys: " + + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen)); + } + lastBloomKey = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; + this.lastCell = cell; + } else { + ByteBuffer br; + int bloomKeyOffset, bloomKeyLen; + switch (bloomType) { + case ROW: + br = cell.getRowBuffer(); + bloomKeyOffset = cell.getRowOffset(); + bloomKeyLen = cell.getRowLength(); + break; + case ROWCOL: + // merge(row, qualifier) + // TODO: could save one buffer copy in case of compound Bloom + // filters when this involves creating a KeyValue + br = generalBloomFilterWriter.createBloomKey(cell.getRowBuffer(), + cell.getRowOffset(), cell.getRowLength(), cell.getQualifierBuffer(), + cell.getQualifierOffset(), cell.getQualifierLength()); + // This should be on heap only here. TODO: Fix it to be done in a + // better way + bloomKeyOffset = 0; + bloomKeyLen = br.capacity(); + break; + default: + throw new IOException("Invalid Bloom filter type: " + bloomType + + " (ROW or ROWCOL expected)"); + } + generalBloomFilterWriter.add(br, bloomKeyOffset, bloomKeyLen); + if (lastBloomKey != null + && generalBloomFilterWriter.getComparator().compareFlatKey(br, + bloomKeyOffset, bloomKeyLen, lastBloomKey_BR, lastBloomKeyOffset, lastBloomKeyLen) <= 0) { + throw new IOException("Non-increasing Bloom keys: " + + ByteBufferUtils.toStringBinary(br, bloomKeyOffset, bloomKeyLen) + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen)); + } + // Have to go with copying + lastBloomKey_BR = br; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; + this.lastCell = cell; } - lastBloomKey = bloomKey; - lastBloomKeyOffset = bloomKeyOffset; - lastBloomKeyLen = bloomKeyLen; - this.lastCell = cell; } } } @@ -917,7 +961,7 @@ public class StoreFile { newKey = !kvComparator.matchingRows(cell, lastDeleteFamilyCell); } if (newKey) { - this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(), + this.deleteFamilyBloomFilterWriter.add(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength()); this.lastDeleteFamilyCell = cell; } @@ -1028,7 +1072,7 @@ public class StoreFile { private final HFile.Reader reader; protected TimeRangeTracker timeRangeTracker = null; protected long sequenceID = -1; - private byte[] lastBloomKey; + private ByteBuffer lastBloomKey; private long deleteFamilyCnt = -1; private boolean bulkLoadResult = false; @@ -1186,8 +1230,7 @@ public class StoreFile { } } - public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, - int rowLen) { + public boolean passesDeleteFamilyBloomFilter(Cell cell) { // Cache Bloom filter as a local variable in case it is set to null by // another thread on an IO error. BloomFilter bloomFilter = this.deleteFamilyBloomFilter; @@ -1205,13 +1248,17 @@ public class StoreFile { if (!bloomFilter.supportsAutoLoading()) { return true; } - return bloomFilter.contains(row, rowOffset, rowLen, null); + if (cell.hasArray()) { + return bloomFilter.contains(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), + null); + } else { + return bloomFilter.contains(cell.getRowBuffer(), cell.getRowOffset(), + cell.getRowLength(), null); + } } catch (IllegalArgumentException e) { - LOG.error("Bad Delete Family bloom filter data -- proceeding without", - e); + LOG.error("Bad Delete Family bloom filter data -- proceeding without", e); setDeleteFamilyBloomFilterFaulty(); } - return true; } @@ -1263,51 +1310,135 @@ public class StoreFile { if (reader.getTrailer().getEntryCount() == 0) return false; + HFileBlock bloomBlock = null; try { - boolean shouldCheckBloom; - ByteBuffer bloom; - if (bloomFilter.supportsAutoLoading()) { - bloom = null; - shouldCheckBloom = true; + Pair res = getBloomBlock(bloomFilter); + MultiByteBuffer bloom = null; + if (res.getFirst() != null) { + bloomBlock = res.getFirst(); + bloom = bloomBlock.getBufferWithoutHeader(); + } + if (res.getSecond()) { + //Wrapping with BB to minimise the code duplicate. + // If this is not needed then we may have to copy from BB to byte[] so that + // Hash.java need not have BB related methods. Going by this approach of wrapping + // the byte[] over BB then we have to introduce BB methods in Hash.java. Compatability issue? + return checkForBloomFilter(ByteBuffer.wrap(row), rowOffset, rowLen, bloomFilter, + ByteBuffer.wrap(key), bloom); + } + } catch (IOException e) { + LOG.error("Error reading bloom filter data -- proceeding without", + e); + setGeneralBloomFilterFaulty(); + } catch (IllegalArgumentException e) { + LOG.error("Bad bloom filter data -- proceeding without", e); + setGeneralBloomFilterFaulty(); + } finally { + reader.returnBlock(bloomBlock); + } + + return true; + } + + private boolean checkForBloomFilter(ByteBuffer row, int rowOffset, int rowLen, + BloomFilter bloomFilter, ByteBuffer key, MultiByteBuffer bloom) { + boolean exists; + + // Whether the primary Bloom key is greater than the last Bloom key + // from the file info. For row-column Bloom filters this is not yet + // a sufficient condition to return false. + boolean keyIsAfterLast = lastBloomKey != null + && bloomFilter.getComparator().compareRows(key, lastBloomKey) > 0; + + if (bloomFilterType == BloomType.ROWCOL) { + // Since a Row Delete is essentially a DeleteFamily applied to all + // columns, a file might be skipped if using row+col Bloom filter. + // In order to ensure this file is included an additional check is + // required looking only for a row bloom. + ByteBuffer rowBloomKey = bloomFilter.createBloomKey(row, rowOffset, rowLen, + null, 0, 0); + + // TODO : for now change key to BR + if (keyIsAfterLast + && bloomFilter.getComparator().compareFlatKey(rowBloomKey, + lastBloomKey) > 0) { + exists = false; } else { - bloom = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, - true); - shouldCheckBloom = bloom != null; + exists = + bloomFilter.contains(key, 0, key.limit(), bloom) || + bloomFilter.contains(rowBloomKey, 0, rowBloomKey.limit(), + bloom); } + } else { + exists = !keyIsAfterLast + && bloomFilter.contains(key, 0, key.limit(), bloom); + } + return exists; + } + + /** + * A method for checking Bloom filters. Called directly from + * StoreFileScanner in case of a multi-column query. + * + * @param row + * @param rowOffset + * @param rowLen + * @param col + * @param colOffset + * @param colLen + * @return True if passes + */ + // TODO : refactor with the above + public boolean passesGeneralBloomFilter(ByteBuffer row, int rowOffset, + int rowLen, ByteBuffer col, int colOffset, int colLen) { + // Cache Bloom filter as a local variable in case it is set to null by + // another thread on an IO error. + BloomFilter bloomFilter = this.generalBloomFilter; + if (bloomFilter == null) { + return true; + } - if (shouldCheckBloom) { - boolean exists; - - // Whether the primary Bloom key is greater than the last Bloom key - // from the file info. For row-column Bloom filters this is not yet - // a sufficient condition to return false. - boolean keyIsAfterLast = lastBloomKey != null - && bloomFilter.getComparator().compareFlatKey(key, lastBloomKey) > 0; - - if (bloomFilterType == BloomType.ROWCOL) { - // Since a Row Delete is essentially a DeleteFamily applied to all - // columns, a file might be skipped if using row+col Bloom filter. - // In order to ensure this file is included an additional check is - // required looking only for a row bloom. - byte[] rowBloomKey = bloomFilter.createBloomKey(row, rowOffset, rowLen, - null, 0, 0); - - if (keyIsAfterLast - && bloomFilter.getComparator().compareFlatKey(rowBloomKey, - lastBloomKey) > 0) { - exists = false; - } else { - exists = - bloomFilter.contains(key, 0, key.length, bloom) || - bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length, - bloom); - } - } else { - exists = !keyIsAfterLast - && bloomFilter.contains(key, 0, key.length, bloom); + ByteBuffer key; + switch (bloomFilterType) { + case ROW: + if (col != null) { + throw new RuntimeException("Row-only Bloom filter called with " + + "column specified"); + } + // This check would never be needed as we don call this in case of ROW bloom. + // row blooms are always formed from scan parameters. But now we will convert all the + // byte[] based row bloom checks to BB + if (rowOffset != 0 || rowLen != row.limit()) { + throw new RuntimeException("For row-only Bloom filters the row " + + "must occupy the whole array"); } + key = row; + break; - return exists; + case ROWCOL: + + key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col, + colOffset, colLen); + break; + + default: + return true; + } + + // Empty file + if (reader.getTrailer().getEntryCount() == 0) + return false; + + HFileBlock bloomBlock = null; + try { + Pair res = getBloomBlock(bloomFilter); + MultiByteBuffer bloom = null; + if (res.getFirst() != null) { + bloomBlock = res.getFirst(); + bloom = bloomBlock.getBufferWithoutHeader(); + } + if (res.getSecond()) { + return checkForBloomFilter(row, rowOffset, rowLen, bloomFilter, key, bloom); } } catch (IOException e) { LOG.error("Error reading bloom filter data -- proceeding without", @@ -1316,11 +1447,32 @@ public class StoreFile { } catch (IllegalArgumentException e) { LOG.error("Bad bloom filter data -- proceeding without", e); setGeneralBloomFilterFaulty(); + } finally { + this.reader.returnBlock(bloomBlock); } return true; } + private Pair getBloomBlock(BloomFilter bloomFilter) throws IOException { + HFileBlock bloomBlock = null; + Pair res = new Pair(); + + boolean shouldCheckBloom = false; + if (bloomFilter.supportsAutoLoading()) { + res.setFirst(null); + shouldCheckBloom = true; + } else { + bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true); + if (bloomBlock != null) { + res.setFirst(bloomBlock); + shouldCheckBloom = bloomBlock != null; + } + } + res.setSecond(shouldCheckBloom); + return res; + } + /** * Checks whether the given scan rowkey range overlaps with the current storefile's * @param scan the scan specification. Used to determine the rowkey range. @@ -1358,7 +1510,9 @@ public class StoreFile { bloomFilterType = BloomType.valueOf(Bytes.toString(b)); } - lastBloomKey = fi.get(LAST_BLOOM_KEY); + if (fi.get(LAST_BLOOM_KEY) != null) { + lastBloomKey = ByteBuffer.wrap(fi.get(LAST_BLOOM_KEY)); + } byte[] cnt = fi.get(DELETE_FAMILY_COUNT); if (cnt != null) { deleteFamilyCnt = Bytes.toLong(cnt); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index a8ee091..2af99b4 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -214,9 +214,10 @@ public class StoreFileScanner implements KeyValueScanner { && (cur.getMvccVersion() > readPt)) { hfs.next(); setCurrentCell(hfs.getKeyValue()); + // TODO : Better to add hasArray here if (this.stopSkippingKVsIfNextRow - && getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(), - cur.getRowLength(), startKV.getRowArray(), startKV.getRowOffset(), + && getComparator().compareRows(cur.getRowBuffer(), cur.getRowOffset(), + cur.getRowLength(), startKV.getRowBuffer(), startKV.getRowOffset(), startKV.getRowLength()) > 0) { return false; } @@ -233,6 +234,10 @@ public class StoreFileScanner implements KeyValueScanner { public void close() { // Nothing to close on HFileScanner? cur = null; + // We have to return the block from which the scanner was closed + if(hfs != null) { + hfs.close(); + } } /** @@ -313,15 +318,20 @@ public class StoreFileScanner implements KeyValueScanner { if (useBloom) { // check ROWCOL Bloom filter first. if (reader.getBloomFilterType() == BloomType.ROWCOL) { - haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(), - kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(), - kv.getQualifierOffset(), kv.getQualifierLength()); + if(kv.hasArray()) { + haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(), kv.getRowOffset(), + kv.getRowLength(), kv.getQualifierArray(), kv.getQualifierOffset(), + kv.getQualifierLength()); + } else { + haveToSeek = reader.passesGeneralBloomFilter(kv.getRowBuffer(), kv.getRowOffset(), + kv.getRowLength(), kv.getQualifierBuffer(), kv.getQualifierOffset(), + kv.getQualifierLength()); + } } else if (this.matcher != null && !matcher.hasNullColumnInQuery() && ((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) { // if there is no such delete family kv in the store file, // then no need to seek. - haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(), - kv.getRowOffset(), kv.getRowLength()); + haveToSeek = reader.passesDeleteFamilyBloomFilter(kv); } } @@ -341,7 +351,7 @@ public class StoreFileScanner implements KeyValueScanner { // a higher timestamp than the max timestamp in this file. We know that // the next point when we have to consider this file again is when we // pass the max timestamp of this file (with the same row/column). - setCurrentCell(KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile)); + setCurrentCell(CellUtil.createFirstOnRowColTS(kv, maxTimestampInFile)); } else { // This will be the case e.g. when we need to seek to the next // row/column, and we don't know exactly what they are, so we set the @@ -359,7 +369,7 @@ public class StoreFileScanner implements KeyValueScanner { // key/value and the store scanner will progress to the next column. This // is obviously not a "real real" seek, but unlike the fake KV earlier in // this method, we want this to be propagated to ScanQueryMatcher. - setCurrentCell(KeyValueUtil.createLastOnRowCol(kv)); + setCurrentCell(CellUtil.createLastOnColumn(kv)); realSeekDone = true; return true; @@ -419,17 +429,14 @@ public class StoreFileScanner implements KeyValueScanner { public boolean seekToPreviousRow(Cell key) throws IOException { try { try { - KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(), - key.getRowLength()); + Cell seekKey = CellUtil.createFirstOnRow(key); + if (seekCount != null) seekCount.incrementAndGet(); - if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(), - seekKey.getKeyLength())) { + if (!hfs.seekBefore(seekKey)) { close(); return false; } - KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue() - .getRowArray(), hfs.getKeyValue().getRowOffset(), hfs.getKeyValue().getRowLength()); - + Cell firstKeyOfPreviousRow = CellUtil.createFirstOnRow(hfs.getKeyValue()); if (seekCount != null) seekCount.incrementAndGet(); if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) { close(); @@ -477,8 +484,8 @@ public class StoreFileScanner implements KeyValueScanner { public boolean backwardSeek(Cell key) throws IOException { seek(key); if (cur == null - || getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(), - cur.getRowLength(), key.getRowArray(), key.getRowOffset(), + || getComparator().compareRows(cur.getRowBuffer(), cur.getRowOffset(), + cur.getRowLength(), key.getRowBuffer(), key.getRowOffset(), key.getRowLength()) > 0) { return seekToPreviousRow(key); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 7ce4e0b..75dfb65 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode; import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; /** @@ -339,6 +338,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner scanner.seek(seekKey); Cell c = scanner.peek(); if (c != null) { + // This is not correct in case of BBKV and KV.. It does not give the correct estimate totalScannersSoughtBytes += CellUtil.estimatedSerializedSizeOf(c); } } @@ -485,18 +485,15 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // only call setRow if the row changes; avoids confusing the query matcher // if scanning intra-row - byte[] row = peeked.getRowArray(); - int offset = peeked.getRowOffset(); - short length = peeked.getRowLength(); - // If limit < 0 and remainingResultSize < 0 we can skip the row comparison because we know // the row has changed. Else it is possible we are still traversing the same row so we // must perform the row comparison. - if ((limit < 0 && remainingResultSize < 0) || matcher.row == null - || !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, - matcher.rowLength)) { - this.countPerRow = 0; - matcher.setRow(row, offset, length); + int rowOffset = peeked.getRowOffset(); + short rowLength = peeked.getRowLength(); + if (limit < 0 || matcher.rowCell == null || !CellUtil.matchingRow(peeked, rowOffset, rowLength, + matcher.rowCell, matcher.rowOffset, matcher.rowLength)) { + this.countPerRow = 0; + matcher.setRow(peeked, rowOffset, rowLength); } Cell cell; @@ -733,14 +730,14 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner if (kv == null) { kv = lastTopKey; } - byte[] row = kv.getRowArray(); - int offset = kv.getRowOffset(); - short length = kv.getRowLength(); - if ((matcher.row == null) || !Bytes.equals(row, offset, length, matcher.row, - matcher.rowOffset, matcher.rowLength)) { + int rowOffset = kv.getRowOffset(); + short rowLength = kv.getRowLength(); + if ((matcher.rowCell == null) + || !CellUtil.matchingRow(kv, rowOffset, rowLength, matcher.rowCell, matcher.rowOffset, + matcher.rowLength)) { this.countPerRow = 0; matcher.reset(); - matcher.setRow(row, offset, length); + matcher.setRow(kv, rowOffset, rowLength); } } @@ -760,7 +757,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner } protected boolean seekToNextRow(Cell kv) throws IOException { - return reseek(KeyValueUtil.createLastOnRow(kv)); + return reseek(CellUtil.createLastOnRow(kv)); } /** diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java index a80c508..969bd81 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java @@ -22,12 +22,12 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.codec.BaseDecoder; import org.apache.hadoop.hbase.codec.BaseEncoder; import org.apache.hadoop.hbase.codec.Codec; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java index 48a982e..51d7425 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java @@ -19,18 +19,19 @@ package org.apache.hadoop.hbase.security.access; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.ByteRange; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.SimpleMutableByteRange; +import org.apache.hadoop.hbase.util.ByteBufferUtils; /** * NOTE: for internal use only by AccessController implementation @@ -62,11 +63,11 @@ class AccessControlFilter extends FilterBase { private User user; private boolean isSystemTable; private Strategy strategy; - private Map cfVsMaxVersions; + private Map cfVsMaxVersions; private int familyMaxVersions; private int currentVersions; - private ByteRange prevFam; - private ByteRange prevQual; + private ByteBuffer prevFam; + private ByteBuffer prevQual; /** * For Writable @@ -75,15 +76,16 @@ class AccessControlFilter extends FilterBase { } AccessControlFilter(TableAuthManager mgr, User ugi, TableName tableName, - Strategy strategy, Map cfVsMaxVersions) { + Strategy strategy, Map cfVsMaxVersions) { authManager = mgr; table = tableName; user = ugi; isSystemTable = tableName.isSystemTable(); this.strategy = strategy; - this.cfVsMaxVersions = cfVsMaxVersions; - this.prevFam = new SimpleMutableByteRange(); - this.prevQual = new SimpleMutableByteRange(); + this.cfVsMaxVersions = new HashMap<>(cfVsMaxVersions.size()); + for(Entry entry: cfVsMaxVersions.entrySet()){ + this.cfVsMaxVersions.put(ByteBuffer.wrap(entry.getKey()), entry.getValue()); + } } @Override @@ -91,21 +93,27 @@ class AccessControlFilter extends FilterBase { if (isSystemTable) { return ReturnCode.INCLUDE; } - if (prevFam.getBytes() == null - || (Bytes.compareTo(prevFam.getBytes(), prevFam.getOffset(), prevFam.getLength(), - cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()) != 0)) { - prevFam.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + int familyOffset = cell.getFamilyOffset(); + byte familyLength = cell.getFamilyLength(); + if (prevFam == null + || (ByteBufferUtils.compareTo(prevFam, 0, prevFam.capacity(), cell.getFamilyBuffer(), + familyOffset, familyLength) != 0)) { + ByteBuffer b = cell.getFamilyBuffer().duplicate(); + b.position(familyOffset).limit(familyOffset + familyLength); + prevFam = b.slice(); // Similar to VisibilityLabelFilter familyMaxVersions = cfVsMaxVersions.get(prevFam); // Family is changed. Just unset curQualifier. - prevQual.unset(); + prevQual = null; } - if (prevQual.getBytes() == null - || (Bytes.compareTo(prevQual.getBytes(), prevQual.getOffset(), - prevQual.getLength(), cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()) != 0)) { - prevQual.set(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()); + int qualOffset = cell.getQualifierOffset(); + int qualLength = cell.getQualifierLength(); + if (prevQual == null + || (ByteBufferUtils.compareTo(prevQual, 0, prevQual.capacity(), cell.getQualifierBuffer(), + qualOffset, qualLength) != 0)) { + ByteBuffer b = cell.getQualifierBuffer().duplicate(); + b.position(qualOffset).limit(qualOffset + qualLength); + prevQual = b.slice(); currentVersions = 0; } currentVersions++; @@ -113,6 +121,7 @@ class AccessControlFilter extends FilterBase { return ReturnCode.SKIP; } // XXX: Compare in place, don't clone + // TODO byte[] family = CellUtil.cloneFamily(cell); byte[] qualifier = CellUtil.cloneQualifier(cell); switch (strategy) { @@ -140,8 +149,8 @@ class AccessControlFilter extends FilterBase { @Override public void reset() throws IOException { - this.prevFam.unset(); - this.prevQual.unset(); + this.prevFam = null; + this.prevQual = null; this.familyMaxVersions = 0; this.currentVersions = 0; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java index b918354..f1ee0ab 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java @@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream; import java.io.DataInput; import java.io.DataInputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.QualifierFilter; import org.apache.hadoop.hbase.filter.RegexStringComparator; +import org.apache.hadoop.hbase.io.ByteBufferInputStream; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; @@ -689,15 +691,24 @@ public class AccessControlLists { return null; } List results = Lists.newArrayList(); - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsIterator = CellUtil.tagsIterator(cell); while (tagsIterator.hasNext()) { Tag tag = tagsIterator.next(); if (tag.getType() == ACL_TAG_TYPE) { // Deserialize the table permissions from the KV - ListMultimap kvPerms = ProtobufUtil.toUsersAndPermissions( - AccessControlProtos.UsersAndPermissions.newBuilder().mergeFrom( - tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()).build()); + int tagOffset = tag.getTagOffset(); + // Avoid this stream.. from that copy to buffer is there!!! + ListMultimap kvPerms; + if (tag.hasArray()) { + kvPerms = ProtobufUtil.toUsersAndPermissions(AccessControlProtos.UsersAndPermissions + .newBuilder().mergeFrom(tag.getBytes(), tagOffset, tag.getTagLength()).build()); + } else { + ByteBuffer tagBB = tag.getBuffer().duplicate(); + tagBB.position(tagOffset).limit(tagOffset + tag.getTagLength()); + // TODO Avoid this stream.. from that copy to buffer is there!!! + kvPerms = ProtobufUtil.toUsersAndPermissions(AccessControlProtos.UsersAndPermissions + .newBuilder().mergeFrom(new ByteBufferInputStream(tagBB)).build()); + } // Are there permissions for this user? List userPerms = kvPerms.get(user.getShortName()); if (userPerms != null) { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 3029413..e7801f5 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -16,6 +16,7 @@ package org.apache.hadoop.hbase.security.access; import java.io.IOException; import java.net.InetAddress; +import java.nio.ByteBuffer; import java.security.PrivilegedExceptionAction; import java.util.Collection; import java.util.HashMap; @@ -30,6 +31,7 @@ import java.util.TreeSet; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -49,6 +51,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagRewriteCell; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -74,6 +77,7 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable; 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.io.ByteBufferInputStream; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.master.MasterServices; @@ -100,11 +104,9 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.Permission.Action; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.SimpleMutableByteRange; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import com.google.common.collect.ArrayListMultimap; @@ -722,16 +724,15 @@ public class AccessController extends BaseMasterAndRegionObserver // This Map is identical to familyMap. The key is a BR rather than byte[]. // It will be easy to do gets over this new Map as we can create get keys over the Cell cf by // new SimpleByteRange(cell.familyArray, cell.familyOffset, cell.familyLen) - Map> familyMap1 = new HashMap>(); + Map> familyMap1 = new HashMap>(); for (Entry> entry : familyMap.entrySet()) { if (entry.getValue() instanceof List) { - familyMap1.put(new SimpleMutableByteRange(entry.getKey()), (List) entry.getValue()); + familyMap1.put(ByteBuffer.wrap(entry.getKey()), (List) entry.getValue()); } } RegionScanner scanner = getRegion(e).getScanner(new Scan(get)); List cells = Lists.newArrayList(); Cell prevCell = null; - ByteRange curFam = new SimpleMutableByteRange(); boolean curColAllVersions = (request == OpType.DELETE); long curColCheckTs = opTs; boolean foundColumn = false; @@ -752,8 +753,10 @@ public class AccessController extends BaseMasterAndRegionObserver continue; } if (colChange && considerCellTs) { - curFam.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); - List cols = familyMap1.get(curFam); + ByteBuffer dup = cell.getFamilyBuffer().duplicate(); + int offset = cell.getFamilyOffset(); + dup.position(offset).limit(offset + cell.getFamilyLength()); + List cols = familyMap1.get(dup); for (Cell col : cols) { // null/empty qualifier is used to denote a Family delete. The TS and delete type // associated with this is applicable for all columns within the family. That is @@ -809,15 +812,15 @@ public class AccessController extends BaseMasterAndRegionObserver List newCells = Lists.newArrayList(); for (Cell cell: e.getValue()) { // Prepend the supplied perms in a new ACL tag to an update list of tags for the cell - List tags = Lists.newArrayList(new Tag(AccessControlLists.ACL_TAG_TYPE, perms)); + List tags = Lists.newArrayList((Tag) new ArrayBackedTag( + AccessControlLists.ACL_TAG_TYPE, perms)); if (cell.getTagsLength() > 0) { - Iterator tagIterator = CellUtil.tagsIterator(cell.getTagsArray(), - cell.getTagsOffset(), cell.getTagsLength()); + Iterator tagIterator = CellUtil.tagsIterator(cell); while (tagIterator.hasNext()) { tags.add(tagIterator.next()); } } - newCells.add(new TagRewriteCell(cell, Tag.fromList(tags))); + newCells.add(new TagRewriteCell(cell, TagUtil.fromList(tags))); } // This is supposed to be safe, won't CME e.setValue(newCells); @@ -838,8 +841,7 @@ public class AccessController extends BaseMasterAndRegionObserver for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { Cell cell = cellScanner.current(); if (cell.getTagsLength() > 0) { - Iterator tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsItr = CellUtil.tagsIterator(cell); while (tagsItr.hasNext()) { if (tagsItr.next().getType() == AccessControlLists.ACL_TAG_TYPE) { throw new AccessDeniedException("Mutation contains cell with reserved type tag"); @@ -1393,9 +1395,9 @@ public class AccessController extends BaseMasterAndRegionObserver AuthResult authResult = permissionGranted(opType, user, env, families, Action.READ); HRegion region = getRegion(env); TableName table = getTableName(region); - Map cfVsMaxVersions = Maps.newHashMap(); + Map cfVsMaxVersions = Maps.newHashMap(); for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) { - cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions()); + cfVsMaxVersions.put(hcd.getName(), hcd.getMaxVersions()); } if (!authResult.isAllowed()) { if (!cellFeaturesEnabled || compatibleEarlyTermination) { @@ -1854,8 +1856,7 @@ public class AccessController extends BaseMasterAndRegionObserver if (oldCell != null) { // Save an object allocation where we can if (oldCell.getTagsLength() > 0) { - Iterator tagIterator = CellUtil.tagsIterator(oldCell.getTagsArray(), - oldCell.getTagsOffset(), oldCell.getTagsLength()); + Iterator tagIterator = CellUtil.tagsIterator(oldCell); while (tagIterator.hasNext()) { Tag tag = tagIterator.next(); if (tag.getType() != AccessControlLists.ACL_TAG_TYPE) { @@ -1867,9 +1868,19 @@ public class AccessController extends BaseMasterAndRegionObserver tags.add(tag); } else { // Merge the perms from the older ACL into the current permission set - ListMultimap kvPerms = ProtobufUtil.toUsersAndPermissions( - AccessControlProtos.UsersAndPermissions.newBuilder().mergeFrom( - tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()).build()); + + int tagOffset = tag.getTagOffset(); + ListMultimap kvPerms; + if (tag.hasArray()) { + kvPerms = ProtobufUtil.toUsersAndPermissions(AccessControlProtos.UsersAndPermissions + .newBuilder().mergeFrom(tag.getBytes(), tagOffset, tag.getTagLength()).build()); + } else { + ByteBuffer tagBB = tag.getBuffer().duplicate(); + tagBB.position(tagOffset).limit(tagOffset + tag.getTagLength()); + // TODO Avoid this stream.. from that copy to buffer is there!!! + kvPerms = ProtobufUtil.toUsersAndPermissions(AccessControlProtos.UsersAndPermissions + .newBuilder().mergeFrom(new ByteBufferInputStream(tagBB)).build()); + } perms.putAll(kvPerms); } } @@ -1880,7 +1891,7 @@ public class AccessController extends BaseMasterAndRegionObserver byte[] aclBytes = mutation.getACL(); if (aclBytes != null) { // Yes, use it - tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE, aclBytes)); + tags.add(new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, aclBytes)); } else { // No, use what we carried forward if (perms != null) { @@ -1890,7 +1901,7 @@ public class AccessController extends BaseMasterAndRegionObserver if (LOG.isTraceEnabled()) { LOG.trace("Carrying forward ACLs from " + oldCell + ": " + perms); } - tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE, + tags.add(new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, ProtobufUtil.toUsersAndPermissions(perms).toByteArray())); } } @@ -1900,7 +1911,7 @@ public class AccessController extends BaseMasterAndRegionObserver return newCell; } - Cell rewriteCell = new TagRewriteCell(newCell, Tag.fromList(tags)); + Cell rewriteCell = new TagRewriteCell(newCell, TagUtil.fromList(tags)); return rewriteCell; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java index 6b9a358..921dcaa 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java @@ -42,6 +42,7 @@ import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; @@ -60,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.OperationStatus; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessControlLists; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -90,7 +92,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService } catch (IOException e) { // We write to a byte array. No Exception can happen. } - LABELS_TABLE_TAGS[0] = new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()); + LABELS_TABLE_TAGS[0] = new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()); } public DefaultVisibilityLabelServiceImpl() { @@ -175,21 +177,22 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService Map> userAuths = new HashMap>(); for (List cells : labelDetails) { for (Cell cell : cells) { - if (Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(), + // TODO based on Cell.hasArray? + if (ByteBufferUtils.equals(cell.getQualifierBuffer(), cell.getQualifierOffset(), cell.getQualifierLength(), LABEL_QUALIFIER, 0, LABEL_QUALIFIER.length)) { labels.put( - Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()), - Bytes.toInt(cell.getRowArray(), cell.getRowOffset())); + ByteBufferUtils.toString(cell.getValueBuffer(), cell.getValueOffset(), cell.getValueLength()), + ByteBufferUtils.getInt(cell.getRowBuffer(), cell.getRowOffset())); } else { // These are user cells who has authorization for this label - String user = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(), + String user = ByteBufferUtils.toString(cell.getQualifierBuffer(), cell.getQualifierOffset(), cell.getQualifierLength()); List auths = userAuths.get(user); if (auths == null) { auths = new ArrayList(); userAuths.put(user, auths); } - auths.add(Bytes.toInt(cell.getRowArray(), cell.getRowOffset())); + auths.add(ByteBufferUtils.getInt(cell.getRowBuffer(), cell.getRowOffset())); } } } @@ -345,7 +348,8 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService scanner.next(results); if (results.isEmpty()) break; Cell cell = results.get(0); - int ordinal = Bytes.toInt(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); + // TODO based on Cell.hasArray? + int ordinal = ByteBufferUtils.getAsInt(cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength()); String label = this.labelsCache.getLabel(ordinal); if (label != null) { auths.add(label); @@ -488,8 +492,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService boolean visibilityTagPresent = false; // Save an object allocation where we can if (cell.getTagsLength() > 0) { - Iterator tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsItr = CellUtil.tagsIterator(cell); while (tagsItr.hasNext()) { boolean includeKV = true; Tag tag = tagsItr.next(); @@ -498,8 +501,12 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService int offset = tag.getTagOffset(); int endOffset = offset + tag.getTagLength(); while (offset < endOffset) { - Pair result = StreamUtils - .readRawVarint32(tag.getBuffer(), offset); + Pair result; + if (tag.hasArray()) { + result = StreamUtils.readRawVarint32(tag.getBytes(), offset); + } else { + result = StreamUtils.readRawVarint32(tag.getBuffer(), offset); + } int currLabelOrdinal = result.getFirst(); if (currLabelOrdinal < 0) { // check for the absence of this label in the Scan Auth labels @@ -616,7 +623,8 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService for (Tag tag : deleteVisTags) { matchFound = false; for (Tag givenTag : putVisTags) { - if (Bytes.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(), + // TODO check for hasArray? + if (ByteBufferUtils.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(), givenTag.getBuffer(), givenTag.getTagOffset(), givenTag.getTagLength())) { matchFound = true; break; @@ -644,7 +652,12 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService int offset = tag.getTagOffset(); int endOffset = offset + tag.getTagLength(); while (offset < endOffset) { - Pair result = StreamUtils.readRawVarint32(tag.getBuffer(), offset); + Pair result; + if (tag.hasArray()) { + result = StreamUtils.readRawVarint32(tag.getBytes(), offset); + } else { + result = StreamUtils.readRawVarint32(tag.getBuffer(), offset); + } tagsOrdinalInSortedOrder.add(result.getFirst()); offset += result.getSecond(); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index f65494c..c5789e3 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -306,8 +307,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements // Indicates that the cell has a the tag which was modified in the src replication cluster Tag tag = pair.getSecond(); if (cellVisibility == null && tag != null) { + assert tag.hasArray(); // May need to store only the first one - cellVisibility = new CellVisibility(Bytes.toString(tag.getBuffer(), tag.getTagOffset(), + cellVisibility = new CellVisibility(Bytes.toString(tag.getBytes(), tag.getTagOffset(), tag.getTagLength())); modifiedTagFound = true; } @@ -335,14 +337,13 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements List updatedCells = new ArrayList(); for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { Cell cell = cellScanner.current(); - List tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + List tags = TagUtil.asList(cell); if (modifiedTagFound) { // Rewrite the tags by removing the modified tags. removeReplicationVisibilityTag(tags); } tags.addAll(visibilityTags); - Cell updatedCell = new TagRewriteCell(cell, Tag.fromList(tags)); + Cell updatedCell = new TagRewriteCell(cell, TagUtil.fromList(tags)); updatedCells.add(updatedCell); } m.getFamilyCellMap().clear(); @@ -437,8 +438,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements // have been modified Tag modifiedTag = null; if (cell.getTagsLength() > 0) { - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), - cell.getTagsOffset(), cell.getTagsLength()); + Iterator tagsIterator = CellUtil.tagsIterator(cell); while (tagsIterator.hasNext()) { Tag tag = tagsIterator.next(); if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) { @@ -452,8 +452,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements return pair; } if (cell.getTagsLength() > 0) { - Iterator tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsItr = CellUtil.tagsIterator(cell); while (tagsItr.hasNext()) { if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) { return pair; @@ -485,8 +484,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements return true; } if (cell.getTagsLength() > 0) { - Iterator tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsItr = CellUtil.tagsIterator(cell); while (tagsItr.hasNext()) { if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) { return false; @@ -698,8 +696,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements // Save an object allocation where we can if (newCell.getTagsLength() > 0) { // Carry forward all other tags - Iterator tagsItr = CellUtil.tagsIterator(newCell.getTagsArray(), - newCell.getTagsOffset(), newCell.getTagsLength()); + Iterator tagsItr = CellUtil.tagsIterator(newCell); while (tagsItr.hasNext()) { Tag tag = tagsItr.next(); if (tag.getType() != TagType.VISIBILITY_TAG_TYPE @@ -709,7 +706,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements } } - Cell rewriteCell = new TagRewriteCell(newCell, Tag.fromList(tags)); + Cell rewriteCell = new TagRewriteCell(newCell, TagUtil.fromList(tags)); return rewriteCell; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java index eb8abbe..72fa02b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java @@ -18,14 +18,13 @@ package org.apache.hadoop.hbase.security.visibility; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Map; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.filter.FilterBase; -import org.apache.hadoop.hbase.util.ByteRange; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.SimpleMutableByteRange; +import org.apache.hadoop.hbase.util.ByteBufferUtils; /** * This Filter checks the visibility expression with each KV against visibility labels associated @@ -35,39 +34,50 @@ import org.apache.hadoop.hbase.util.SimpleMutableByteRange; class VisibilityLabelFilter extends FilterBase { private final VisibilityExpEvaluator expEvaluator; - private final Map cfVsMaxVersions; - private final ByteRange curFamily; - private final ByteRange curQualifier; + private final Map cfVsMaxVersions; + private ByteBuffer curFamily; + private int curFamilyOffset; + private int curFamilyLength; + private ByteBuffer curQualifier; + private int curQualifierOffset; + private int curQualifierLength; private int curFamilyMaxVersions; private int curQualMetVersions; public VisibilityLabelFilter(VisibilityExpEvaluator expEvaluator, - Map cfVsMaxVersions) { + Map cfVsMaxVersions) { this.expEvaluator = expEvaluator; this.cfVsMaxVersions = cfVsMaxVersions; - this.curFamily = new SimpleMutableByteRange(); - this.curQualifier = new SimpleMutableByteRange(); + this.curFamily = null; + this.curQualifier = null; } @Override public ReturnCode filterKeyValue(Cell cell) throws IOException { - if (curFamily.getBytes() == null - || (Bytes.compareTo(curFamily.getBytes(), curFamily.getOffset(), curFamily.getLength(), - cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()) != 0)) { - curFamily.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); + if (curFamily == null || (ByteBufferUtils.compareTo(curFamily, curFamilyOffset, curFamilyLength, + cell.getFamilyBuffer(), cell.getFamilyOffset(), cell.getFamilyLength()) != 0)) { + curFamily = cell.getFamilyBuffer(); + curFamilyOffset = cell.getFamilyOffset(); + curFamilyLength = cell.getFamilyLength(); + //curFamily.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()); // For this family, all the columns can have max of curFamilyMaxVersions versions. No need to // consider the older versions for visibility label check. // Ideally this should have been done at a lower layer by HBase (?) - curFamilyMaxVersions = cfVsMaxVersions.get(curFamily); + ByteBuffer duplicate = curFamily.duplicate(); + duplicate.position(curFamilyOffset).limit(curFamilyOffset + curFamilyLength); + curFamilyMaxVersions = cfVsMaxVersions.get(duplicate); // Family is changed. Just unset curQualifier. - curQualifier.unset(); + curQualifier = null; + //curQualifier.unset(); } - if (curQualifier.getBytes() == null - || (Bytes.compareTo(curQualifier.getBytes(), curQualifier.getOffset(), - curQualifier.getLength(), cell.getQualifierArray(), cell.getQualifierOffset(), + if (curQualifier == null || (ByteBufferUtils.compareTo(curQualifier, curQualifierOffset, + curQualifierLength, cell.getQualifierBuffer(), cell.getQualifierOffset(), cell.getQualifierLength()) != 0)) { - curQualifier.set(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()); + curQualifier = cell.getQualifierBuffer(); + curQualifierOffset = cell.getQualifierOffset(); + curQualifierLength = cell.getQualifierLength(); + /*curQualifier.set(cell.getQualifierArray(), cell.getQualifierOffset(), + cell.getQualifierLength());*/ curQualMetVersions = 0; } curQualMetVersions++; @@ -80,8 +90,12 @@ class VisibilityLabelFilter extends FilterBase { @Override public void reset() throws IOException { - this.curFamily.unset(); - this.curQualifier.unset(); + this.curFamily = null; + this.curQualifier = null; + this.curFamilyOffset = 0; + this.curFamilyLength = 0; + this.curQualifierOffset = 0; + this.curQualifierLength = 0; this.curFamilyMaxVersions = 0; this.curQualMetVersions = 0; } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java index aca4994..5614065 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java @@ -24,10 +24,12 @@ import java.util.UUID; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; @@ -79,7 +81,8 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint { byte[] modifiedVisExpression = visibilityLabelsService .encodeVisibilityForReplication(visTags, serializationFormat); if (modifiedVisExpression != null) { - nonVisTags.add(new Tag(TagType.STRING_VIS_TAG_TYPE, modifiedVisExpression)); + nonVisTags.add(new ArrayBackedTag(TagType.STRING_VIS_TAG_TYPE, + modifiedVisExpression)); } } catch (Exception ioe) { LOG.error( @@ -92,7 +95,7 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint { continue; } // Recreate the cell with the new tags and the existing tags - Cell newCell = new TagRewriteCell(cell, Tag.fromList(nonVisTags)); + Cell newCell = new TagRewriteCell(cell, TagUtil.fromList(nonVisTags)); newEdit.add(newCell); } else { newEdit.add(cell); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityScanDeleteTracker.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityScanDeleteTracker.java index 80e1d5d..5209db3 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityScanDeleteTracker.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityScanDeleteTracker.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.regionserver.ScanDeleteTracker; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -90,7 +91,7 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker { } // new column, or more general delete type if (deleteBuffer != null) { - if (Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength, delCell.getQualifierArray(), + if (ByteBufferUtils.compareTo(deleteBuffer, deleteOffset, deleteLength, delCell.getQualifierBuffer(), qualifierOffset, qualifierLength) != 0) { // A case where there are deletes for a column qualifier but there are // no corresponding puts for them. Rare case. @@ -107,7 +108,7 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker { visiblityTagsDeleteColumnVersion = null; } } - deleteBuffer = delCell.getQualifierArray(); + deleteBuffer = delCell.getQualifierBuffer(); deleteOffset = qualifierOffset; deleteLength = qualifierLength; deleteType = type; @@ -239,8 +240,8 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker { } } if (deleteBuffer != null) { - int ret = Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength, - cell.getQualifierArray(), qualifierOffset, qualifierLength); + int ret = ByteBufferUtils.compareTo(deleteBuffer, deleteOffset, deleteLength, + cell.getQualifierBuffer(), qualifierOffset, qualifierLength); if (ret == 0) { if (deleteType == KeyValue.Type.DeleteColumn.getCode()) { @@ -294,7 +295,7 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker { visiblityTagsDeleteColumnVersion = null; } else { throw new IllegalStateException("isDeleted failed: deleteBuffer=" - + Bytes.toStringBinary(deleteBuffer, deleteOffset, deleteLength) + ", qualifier=" + + ByteBufferUtils.toStringBinary(deleteBuffer, deleteOffset, deleteLength) + ", qualifier=" + Bytes.toStringBinary(cell.getQualifierArray(), qualifierOffset, qualifierLength) + ", timestamp=" + timestamp + ", comparison result: " + ret); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java index ebff5ff..66f3114 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.TagType.VISIBILITY_TAG_TYPE; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -34,13 +35,14 @@ import java.util.Set; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.util.StreamUtils; @@ -58,11 +60,10 @@ import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.Operator; -import org.apache.hadoop.hbase.util.ByteRange; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.SimpleMutableByteRange; import org.apache.hadoop.util.ReflectionUtils; import com.google.protobuf.InvalidProtocolBufferException; @@ -78,7 +79,7 @@ public class VisibilityUtils { public static final String VISIBILITY_LABEL_GENERATOR_CLASS = "hbase.regionserver.scan.visibility.label.generator.class"; public static final String SYSTEM_LABEL = "system"; - public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG = new Tag( + public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG = new ArrayBackedTag( TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE, VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG_VAL); private static final String COMMA = ","; @@ -246,12 +247,15 @@ public class VisibilityUtils { public static Byte extractVisibilityTags(Cell cell, List tags) { Byte serializationFormat = null; if (cell.getTagsLength() > 0) { - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsIterator = CellUtil.tagsIterator(cell); while (tagsIterator.hasNext()) { Tag tag = tagsIterator.next(); if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) { - serializationFormat = tag.getBuffer()[tag.getTagOffset()]; + if (tag.hasArray()) { + serializationFormat = tag.getBytes()[tag.getTagOffset()]; + } else { + serializationFormat = ByteBufferUtils.getByte(tag.getBuffer(), tag.getTagOffset()); + } } else if (tag.getType() == VISIBILITY_TAG_TYPE) { tags.add(tag); } @@ -276,12 +280,15 @@ public class VisibilityUtils { List nonVisTags) { Byte serializationFormat = null; if (cell.getTagsLength() > 0) { - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsIterator = CellUtil.tagsIterator(cell); while (tagsIterator.hasNext()) { Tag tag = tagsIterator.next(); if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) { - serializationFormat = tag.getBuffer()[tag.getTagOffset()]; + if (tag.hasArray()) { + serializationFormat = tag.getBytes()[tag.getTagOffset()]; + } else { + serializationFormat = ByteBufferUtils.getByte(tag.getBuffer(), tag.getTagOffset()); + } } else if (tag.getType() == VISIBILITY_TAG_TYPE) { visTags.add(tag); } else { @@ -297,8 +304,7 @@ public class VisibilityUtils { if (cell.getTagsLength() == 0) { return false; } - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsIterator = CellUtil.tagsIterator(cell); while (tagsIterator.hasNext()) { Tag tag = tagsIterator.next(); if (tag.getType() == VISIBILITY_TAG_TYPE) { @@ -310,9 +316,9 @@ public class VisibilityUtils { public static Filter createVisibilityLabelFilter(HRegion region, Authorizations authorizations) throws IOException { - Map cfVsMaxVersions = new HashMap(); + Map cfVsMaxVersions = new HashMap(); for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) { - cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions()); + cfVsMaxVersions.put(ByteBuffer.wrap(hcd.getName()), hcd.getMaxVersions()); } VisibilityLabelService vls = VisibilityLabelServiceManager.getInstance() .getVisibilityLabelService(); @@ -358,7 +364,7 @@ public class VisibilityUtils { if (node.isSingleNode()) { getLabelOrdinals(node, labelOrdinals, auths, checkAuths, ordinalProvider); writeLabelOrdinalsToStream(labelOrdinals, dos); - tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray())); + tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray())); baos.reset(); } else { NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node; @@ -366,14 +372,14 @@ public class VisibilityUtils { for (ExpressionNode child : nlNode.getChildExps()) { getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider); writeLabelOrdinalsToStream(labelOrdinals, dos); - tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray())); + tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray())); baos.reset(); labelOrdinals.clear(); } } else { getLabelOrdinals(nlNode, labelOrdinals, auths, checkAuths, ordinalProvider); writeLabelOrdinalsToStream(labelOrdinals, dos); - tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray())); + tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray())); baos.reset(); } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java index 163852c..10c81d2 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java @@ -53,8 +53,19 @@ public interface BloomFilter extends BloomFilterBase { * is supported. * @return true if matched by bloom, false if not */ - boolean contains(byte [] buf, int offset, int length, ByteBuffer bloom); - + boolean contains(byte[] buf, int offset, int length, MultiByteBuffer bloom); + + /** + * Check if the specified key is contained in the bloom filter. + * + * @param buf data to check for existence of + * @param offset offset into the data + * @param length length of the data + * @param bloom bloom filter data to search. This can be null if auto-loading + * is supported. + * @return true if matched by bloom, false if not + */ + boolean contains(ByteBuffer buf, int offset, int length, MultiByteBuffer bloom); /** * @return true if this Bloom filter can automatically load its data * and thus allows a null byte buffer to be passed to contains() diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java index 3b9ca9a..12b6a5e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.KeyValue.KVComparator; @@ -48,6 +50,11 @@ public interface BloomFilterBase { */ byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, byte[] qualBuf, int qualOffset, int qualLen); + /** + * Create a key for a row-column Bloom filter. + */ + ByteBuffer createBloomKey(ByteBuffer rowBuf, int rowOffset, int rowLen, + ByteBuffer qualBuf, int qualOffset, int qualLen); /** * @return Bloom key comparator diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java index aa7f503..0c71fec 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.io.Writable; @@ -58,5 +60,14 @@ public interface BloomFilterWriter extends BloomFilterBase { * @param len length of the data to be added */ void add(byte[] buf, int offset, int len); + + /** + * Add the specified binary to the bloom filter. + * + * @param buf data to be added to the bloom + * @param offset offset into the data to be added + * @param len length of the data to be added + */ + void add(ByteBuffer buf, int offset, int len); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java index 56c3776..40c2a90 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java @@ -96,7 +96,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { /** Max Keys expected for the bloom */ protected int maxKeys; /** Bloom bits */ - protected ByteBuffer bloom; + protected MultiByteBuffer bloom; /** Record separator for the Bloom filter statistics human-readable string */ public static final String STATS_RECORD_SEP = "; "; @@ -335,7 +335,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { if (this.bloom != null) { throw new IllegalArgumentException("can only create bloom once."); } - this.bloom = ByteBuffer.allocate((int)this.byteSize); + this.bloom = new MultiByteBuffer(ByteBuffer.allocate((int)this.byteSize)); assert this.bloom.hasArray(); } @@ -384,6 +384,23 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { ++this.keyCount; } + + @Override + public void add(ByteBuffer buf, int offset, int len) { + /* + * For faster hashing, use combinatorial generation + * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + */ + int hash1 = this.hash.hash(buf, offset, len, 0); + int hash2 = this.hash.hash(buf, offset, len, hash1); + + for (int i = 0; i < this.hashCount; i++) { + long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8)); + set(hashLoc); + } + + ++this.keyCount; + } /** Should only be used in tests */ boolean contains(byte [] buf) { @@ -397,12 +414,30 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { /** Should only be used in tests */ boolean contains(byte[] buf, ByteBuffer bloom) { - return contains(buf, 0, buf.length, bloom); + return contains(buf, 0, buf.length, new MultiByteBuffer(bloom)); } @Override public boolean contains(byte[] buf, int offset, int length, - ByteBuffer theBloom) { + MultiByteBuffer theBloom) { + if (theBloom == null) { + // In a version 1 HFile Bloom filter data is stored in a separate meta + // block which is loaded on demand, but in version 2 it is pre-loaded. + // We want to use the same API in both cases. + theBloom = bloom; + } + + if (theBloom.limit() != byteSize) { + throw new IllegalArgumentException("Bloom does not match expected size:" + + " theBloom.limit()=" + theBloom.limit() + ", byteSize=" + byteSize); + } + + return contains(buf, offset, length, theBloom, 0, (int) byteSize, hash, hashCount); + } + + @Override + public boolean contains(ByteBuffer buf, int offset, int length, + MultiByteBuffer theBloom) { if (theBloom == null) { // In a version 1 HFile Bloom filter data is stored in a separate meta // block which is loaded on demand, but in version 2 it is pre-loaded. @@ -419,7 +454,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { } public static boolean contains(byte[] buf, int offset, int length, - ByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash, + MultiByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash, int hashCount) { int hash1 = hash.hash(buf, offset, length, 0); @@ -447,7 +482,37 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { } return true; } + + public static boolean contains(ByteBuffer buf, int offset, int length, + MultiByteBuffer bloomBuf, int bloomOffset, int bloomSize, Hash hash, + int hashCount) { + int hash1 = hash.hash(buf, offset, length, 0); + int hash2 = hash.hash(buf, offset, length, hash1); + int bloomBitSize = bloomSize << 3; + + if (randomGeneratorForTest == null) { + // Production mode. + int compositeHash = hash1; + for (int i = 0; i < hashCount; i++) { + int hashLoc = Math.abs(compositeHash % bloomBitSize); + compositeHash += hash2; + if (!get(hashLoc, bloomBuf, bloomOffset)) { + return false; + } + } + } else { + // Test mode with "fake lookups" to estimate "ideal false positive rate". + for (int i = 0; i < hashCount; i++) { + int hashLoc = randomGeneratorForTest.nextInt(bloomBitSize); + if (!get(hashLoc, bloomBuf, bloomOffset)){ + return false; + } + } + } + return true; + } + //--------------------------------------------------------------------------- /** Private helpers */ @@ -470,7 +535,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { * @param pos index of bit * @return true if bit at specified index is 1, false if 0. */ - static boolean get(int pos, ByteBuffer bloomBuf, int bloomOffset) { + static boolean get(int pos, MultiByteBuffer bloomBuf, int bloomOffset) { int bytePos = pos >> 3; //pos / 8 int bitPos = pos & 0x7; //pos % 8 // TODO access this via Util API which can do Unsafe access if possible(?) @@ -527,7 +592,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { // folding done, only use a subset of this array this.bloom.rewind(); this.bloom.limit(newByteSize); - this.bloom = this.bloom.slice(); + //this.bloom = this.bloom.slice(); this.byteSize = newByteSize; this.maxKeys = newMaxKeys; } @@ -624,6 +689,25 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen); return result; } + + /** + * {@inheritDoc} + * Just concatenate row and column by default. May return the original row + * buffer if the column qualifier is empty. + */ + @Override + public ByteBuffer createBloomKey(ByteBuffer rowBuf, int rowOffset, int rowLen, + ByteBuffer qualBuf, int qualOffset, int qualLen) { + // Optimize the frequent case when only the row is provided. + if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.limit()) + return rowBuf; + + ByteBuffer result = ByteBuffer.allocate(rowLen + qualLen); + ByteBufferUtils.copyFromBufferToBuffer(result, rowBuf, rowOffset, 0, rowLen); + if (qualLen > 0) + ByteBufferUtils.copyFromBufferToBuffer(result, qualBuf, qualOffset, rowLen, qualLen); + return result; + } @Override public KVComparator getComparator() { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java index beda805..13eaa6c 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java @@ -23,8 +23,9 @@ import java.io.DataInput; import java.io.IOException; import java.nio.ByteBuffer; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.KeyValue.KVComparator; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer; import org.apache.hadoop.hbase.io.hfile.HFile; @@ -83,8 +84,9 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase } @Override + // TODO : Duplicate APIs for contains public boolean contains(byte[] key, int keyOffset, int keyLength, - ByteBuffer bloom) { + MultiByteBuffer bloom) { // We try to store the result in this variable so we can update stats for // testing, but when an error happens, we log a message and return. boolean result; @@ -107,10 +109,11 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase + Bytes.toStringBinary(key, keyOffset, keyLength), ex); } - ByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); + MultiByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); result = ByteBloomFilter.contains(key, keyOffset, keyLength, bloomBuf, bloomBlock.headerSize(), bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); + reader.returnBlock(bloomBlock); } if (numQueriesPerChunk != null && block >= 0) { @@ -176,4 +179,45 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase return sb.toString(); } + @Override + public boolean contains(ByteBuffer key, int keyOffset, int keyLength, MultiByteBuffer bloom) { + // We try to store the result in this variable so we can update stats for + // testing, but when an error happens, we log a message and return. + boolean result; + + int block = index.rootBlockContainingKey(key, keyOffset, + keyLength); + if (block < 0) { + result = false; // This key is not in the file. + } else { + HFileBlock bloomBlock; + try { + // We cache the block and use a positional read. + bloomBlock = reader.readBlock(index.getRootBlockOffset(block), + index.getRootBlockDataSize(block), true, true, false, true, + BlockType.BLOOM_CHUNK, null); + } catch (IOException ex) { + // The Bloom filter is broken, turn it off. + throw new IllegalArgumentException( + "Failed to load Bloom block for key " + + ByteBufferUtils.toStringBinary(key, keyOffset, keyLength), ex); + } + + MultiByteBuffer bloomBuf = bloomBlock.getBufferReadOnly(); + result = ByteBloomFilter.contains(key, keyOffset, keyLength, + bloomBuf, bloomBlock.headerSize(), + bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); + reader.returnBlock(bloomBlock); + } + + if (numQueriesPerChunk != null && block >= 0) { + // Update statistics. Only used in unit tests. + ++numQueriesPerChunk[block]; + if (result) + ++numPositivesPerChunk[block]; + } + + return result; + } + } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java index af9fa00..160b78e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java @@ -19,9 +19,14 @@ package org.apache.hadoop.hbase.util; +import java.nio.ByteBuffer; + import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ByteBufferBackedKeyValue; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; +import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValueUtil; @InterfaceAudience.Private @@ -70,6 +75,7 @@ public class CompoundBloomFilterBase implements BloomFilterBase { } private static final byte[] DUMMY = new byte[0]; + private static final ByteBuffer DUMMY_BUFFER = (ByteBuffer.wrap(DUMMY)); /** * Prepare an ordered pair of row and qualifier to be compared using @@ -88,6 +94,28 @@ public class CompoundBloomFilterBase implements BloomFilterBase { qualifier, qoffset, qlength); return kv.getKey(); } + + /** + * Prepare an ordered pair of row and qualifier to be compared using + * KeyValue.KeyComparator. This is only used for row-column Bloom + * filters. + */ + @Override + public ByteBuffer createBloomKey(ByteBuffer row, int roffset, int rlength, + ByteBuffer qualifier, int qoffset, int qlength) { + if (qualifier == null) + qualifier = DUMMY_BUFFER; + + // Make sure this does not specify a timestamp so that the default maximum + // (most recent) timestamp is used. + ByteBufferBackedKeyValue kv = new ByteBufferBackedKeyValue(row, roffset, rlength, DUMMY_BUFFER, + 0, 0, qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0); + return kv.getKey(); + /*KeyValue kv = (KeyValue) ByteBufferBackedKeyValueUtil.createFirstOnRow(row, roffset, + rlength, DUMMY_BUFFER, 0, 0, qualifier, qoffset, qlength); + // Here we need BB - hence wrapping + return ByteBuffer.wrap(kv.getKey());*/ + } @Override public KVComparator getComparator() { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java index d436a98..5f00114 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterWriter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.util; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.LinkedList; import java.util.Queue; @@ -69,6 +70,9 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase /** The first key in the current Bloom filter chunk. */ private byte[] firstKeyInChunk = null; + + // TODO : Better name + private ByteBuffer firstKeyInChunk_BR = null; private HFileBlockIndex.BlockIndexWriter bloomBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter(); @@ -118,7 +122,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase return; } - if (firstKeyInChunk == null) { + if (firstKeyInChunk == null && firstKeyInChunk_BR == null) { throw new NullPointerException("Trying to enqueue a chunk, " + "but first key is null: closing=" + closing + ", keyCount=" + chunk.getKeyCount() + ", maxKeys=" + chunk.getMaxKeys()); @@ -127,7 +131,12 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase ReadyChunk readyChunk = new ReadyChunk(); readyChunk.chunkId = numChunks - 1; readyChunk.chunk = chunk; - readyChunk.firstKey = firstKeyInChunk; + if (firstKeyInChunk != null) { + readyChunk.firstKey = firstKeyInChunk; + } else { + // TODO : check in case of offheap cases what to be done here + readyChunk.firstKey = firstKeyInChunk_BR.array(); + } readyChunks.add(readyChunk); long prevMaxKeys = chunk.getMaxKeys(); @@ -146,6 +155,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase totalByteSize += chunk.getByteSize(); firstKeyInChunk = null; + firstKeyInChunk_BR = null; prevChunk = chunk; chunk = null; } @@ -274,4 +284,43 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase public boolean getCacheOnWrite() { return cacheOnWrite; } + + @Override + public void add(ByteBuffer bloomKey, int keyOffset, int keyLength) { + if (bloomKey == null) + throw new NullPointerException(); + + enqueueReadyChunk(false); + + if (chunk == null) { + if (firstKeyInChunk_BR != null) { + throw new IllegalStateException("First key in chunk already set: " + + ByteBufferUtils.toStringBinary(firstKeyInChunk_BR)); + } + // Two levels of copy needed if firstKeyInCheck is not an BR + ByteBuffer br = ByteBufferUtils.copyOfRange(bloomKey, keyOffset, keyOffset + keyLength); + firstKeyInChunk_BR = br; + + if (prevChunk == null) { + // First chunk + chunk = ByteBloomFilter.createBySize(chunkByteSize, errorRate, + hashType, maxFold); + } else { + // Use the same parameters as the last chunk, but a new array and + // a zero key count. + chunk = prevChunk.createAnother(); + } + + if (chunk.getKeyCount() != 0) { + throw new IllegalStateException("keyCount=" + chunk.getKeyCount() + + " > 0"); + } + + chunk.allocBloom(); + ++numChunks; + } + + chunk.add(bloomKey, keyOffset, keyLength); + ++totalKeyCount; + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java index f0d1e67..c7a3d36 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.codehaus.jackson.map.ObjectMapper; @@ -318,12 +319,13 @@ public class WALPrettyPrinter { stringMap.put("vlen", cell.getValueLength()); if (cell.getTagsLength() > 0) { List tagsString = new ArrayList(); - Iterator tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsIterator = CellUtil.tagsIterator(cell); while (tagsIterator.hasNext()) { Tag tag = tagsIterator.next(); - tagsString.add((tag.getType()) + ":" - + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength())); + tagsString.add((tag.getType()) + + ":" + + ByteBufferUtils.toStringBinary(tag.getBuffer(), tag.getTagOffset(), + tag.getTagLength())); } stringMap.put("tag", tagsString); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java index 2e7afa5..ef43b44 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -1218,10 +1218,13 @@ public class PerformanceEvaluation extends Configured implements Tool { Result r = null; int count = 0; ResultScanner s = this.table.getScanner(scan); + long currentTime = System.currentTimeMillis(); + //LOG.info("Starting the scanner "+currentTime); for (; (r = s.next()) != null;) { updateValueSize(r); count++; } + LOG.info("Time taken to fetch "+count+" rows is "+(System.currentTimeMillis() - currentTime)); if (i % 100 == 0) { LOG.info(String.format("Scan for key range %s - %s returned %s rows", Bytes.toString(startAndStopRow.getFirst()), @@ -1234,7 +1237,7 @@ public class PerformanceEvaluation extends Configured implements Tool { protected abstract Pair getStartAndStopRow(); protected Pair generateStartAndStopRows(int maxRange) { - int start = this.rand.nextInt(Integer.MAX_VALUE) % opts.totalRows; + int start = 5000000 % opts.totalRows; int stop = start + maxRange; return new Pair(format(start), format(stop)); } @@ -1314,17 +1317,21 @@ public class PerformanceEvaluation extends Configured implements Tool { if (opts.filterAll) { get.setFilter(new FilterAllFilter()); } + get.setConsistency(consistency); if (LOG.isTraceEnabled()) LOG.trace(get.toString()); if (opts.multiGet > 0) { this.gets.add(get); if (this.gets.size() == opts.multiGet) { + long currentTime = System.currentTimeMillis(); Result [] rs = this.table.get(this.gets); + LOG.info("Time taken to fetch "+rs.length+" rows is "+(System.currentTimeMillis() - currentTime)); updateValueSize(rs); this.gets.clear(); } } else { updateValueSize(this.table.get(get)); + } } @@ -1358,7 +1365,7 @@ public class PerformanceEvaluation extends Configured implements Tool { byte[] tag = generateData(this.rand, TAG_LENGTH); Tag[] tags = new Tag[opts.noOfTags]; for (int n = 0; n < opts.noOfTags; n++) { - Tag t = new Tag((byte) n, tag); + Tag t = new ArrayBackedTag((byte) n, tag); tags[n] = t; } KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP, @@ -1437,7 +1444,7 @@ public class PerformanceEvaluation extends Configured implements Tool { byte[] tag = generateData(this.rand, TAG_LENGTH); Tag[] tags = new Tag[opts.noOfTags]; for (int n = 0; n < opts.noOfTags; n++) { - Tag t = new Tag((byte) n, tag); + Tag t = new ArrayBackedTag((byte) n, tag); tags[n] = t; } KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP, @@ -1465,11 +1472,16 @@ public class PerformanceEvaluation extends Configured implements Tool { byte[] value = generateData(this.rand, getValueLength(this.rand)); Scan scan = constructScan(value); ResultScanner scanner = null; + long currentTime = System.currentTimeMillis(); + LOG.info("Starting the scanner "+currentTime); + int count = 0; try { scanner = this.table.getScanner(scan); for (Result r = null; (r = scanner.next()) != null;) { updateValueSize(r); + count++; } + LOG.info("Time taken to fetch "+count+" rows is "+(System.currentTimeMillis() - currentTime)); } finally { if (scanner != null) scanner.close(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java index 0f0baff..fb563f1 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; @@ -179,8 +180,8 @@ public class TestTableSnapshotScanner { Cell cell = scanner.current(); //assert that all Cells in the Result have the same key - Assert.assertEquals(0, Bytes.compareTo(row, 0, row.length, - cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())); + Assert.assertEquals(0, ByteBufferUtils.compareTo(row, 0, row.length, + cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength())); } for (int j = 0; j < FAMILIES.length; j++) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index 9d448b4..f3f7ae4 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -1618,7 +1618,7 @@ public class TestFilter { int i = 0; for (boolean done = true; done; i++) { done = NextState.hasMoreValues(scanner.next(results)); - Arrays.sort(results.toArray(new KeyValue[results.size()]), + Arrays.sort(results.toArray(new Cell[results.size()]), KeyValue.COMPARATOR); LOG.info("counter=" + i + ", " + results); if (results.isEmpty()) break; @@ -1640,7 +1640,7 @@ public class TestFilter { int i = 0; for (boolean done = true; done; i++) { done = NextState.hasMoreValues(scanner.next(results)); - Arrays.sort(results.toArray(new KeyValue[results.size()]), + Arrays.sort(results.toArray(new Cell[results.size()]), KeyValue.COMPARATOR); LOG.info("counter=" + i + ", " + results); if(results.isEmpty()) break; @@ -1662,7 +1662,7 @@ public class TestFilter { int idx = 0; for (boolean done = true; done; row++) { done = NextState.hasMoreValues(scanner.next(results)); - Arrays.sort(results.toArray(new KeyValue[results.size()]), + Arrays.sort(results.toArray(new Cell[results.size()]), KeyValue.COMPARATOR); if(results.isEmpty()) break; assertTrue("Scanned too many keys! Only expected " + kvs.length + diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java index e527ca8..3c0f949 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java @@ -80,7 +80,7 @@ public class TestInclusiveStopFilter { assertTrue("FilterAllRemaining", filter.filterAllRemaining()); assertFalse("FilterNotNull", filter.filterRow()); - assertFalse("Filter a null", filter.filterRowKey(null, 0, 0)); + assertFalse("Filter a null", filter.filterRowKey((byte[])null, 0, 0)); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java index 55f8dda..f5101c7 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestByteBufferOutputStream.java @@ -46,4 +46,4 @@ public class TestByteBufferOutputStream { return bbos.getByteBuffer(); } } -} \ No newline at end of file +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java index 18595a8..77113bb 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java @@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.LastOnColumnCell; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; @@ -127,7 +127,7 @@ public class TestHalfStoreFileReader { Cell curr; do { curr = scanner.getKeyValue(); - KeyValue reseekKv = + Cell reseekKv = getLastOnCol(curr); int ret = scanner.reseekTo(reseekKv); assertTrue("reseek to returned: " + ret, ret > 0); @@ -227,11 +227,10 @@ public class TestHalfStoreFileReader { return scanner.getKeyValue(); } - private KeyValue getLastOnCol(Cell curr) { - return KeyValueUtil.createLastOnRow( - curr.getRowArray(), curr.getRowOffset(), curr.getRowLength(), - curr.getFamilyArray(), curr.getFamilyOffset(), curr.getFamilyLength(), - curr.getQualifierArray(), curr.getQualifierOffset(), curr.getQualifierLength()); + private Cell getLastOnCol(Cell curr) { + return new LastOnColumnCell(curr.getRowBuffer(), curr.getRowOffset(), curr.getRowLength(), + curr.getFamilyBuffer(), curr.getFamilyOffset(), curr.getFamilyLength(), + curr.getQualifierBuffer(), curr.getQualifierOffset(), curr.getQualifierLength()); } static final int SIZE = 1000; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java index cabb67f..d32ac24 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFileBlock.Writer.BufferGrabbingByteArrayOutputStream; @@ -42,7 +43,9 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.hbase.util.test.RedundantKVGenerator; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -115,10 +118,10 @@ public class TestDataBlockEncoders { } else { byte[] metaValue1 = Bytes.toBytes("metaValue1"); byte[] metaValue2 = Bytes.toBytes("metaValue2"); - kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1, - metaValue1) })); - kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1, - metaValue2) })); + kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new ArrayBackedTag( + (byte) 1, metaValue1) })); + kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new ArrayBackedTag( + (byte) 1, metaValue2) })); } testEncodersOnDataset(kvList, includesMemstoreTS, includesTags); } @@ -139,10 +142,10 @@ public class TestDataBlockEncoders { if (includesTags) { byte[] metaValue1 = Bytes.toBytes("metaValue1"); byte[] metaValue2 = Bytes.toBytes("metaValue2"); - kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1, - metaValue1) })); - kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1, - metaValue2) })); + kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new ArrayBackedTag( + (byte) 1, metaValue1) })); + kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new ArrayBackedTag( + (byte) 1, metaValue2) })); } else { kvList.add(new KeyValue(row, family, qualifier, -1l, Type.Put, value)); kvList.add(new KeyValue(row, family, qualifier, -2l, Type.Put, value)); @@ -257,8 +260,8 @@ public class TestDataBlockEncoders { int i = 0; do { KeyValue expectedKeyValue = sampleKv.get(i); - ByteBuffer keyValue = seeker.getKeyValueBuffer(); - if (0 != Bytes.compareTo(keyValue.array(), keyValue.arrayOffset(), keyValue.limit(), + ByteBuffer keyValue = (seeker.getKeyValueBuffer()); + if (0 != ByteBufferUtils.compareTo(keyValue, 0, keyValue.limit(), expectedKeyValue.getBuffer(), expectedKeyValue.getOffset(), expectedKeyValue.getLength())) { @@ -276,7 +279,7 @@ public class TestDataBlockEncoders { fail(String.format("next() produces wrong results " + "encoder: %s i: %d commonPrefix: %d" + "\n expected %s\n actual %s", encoder .toString(), i, commonPrefix, Bytes.toStringBinary(expectedKeyValue.getBuffer(), - expectedKeyValue.getOffset(), expectedKeyValue.getLength()), Bytes + expectedKeyValue.getOffset(), expectedKeyValue.getLength()), ByteBufferUtils .toStringBinary(keyValue))); } i++; @@ -299,7 +302,7 @@ public class TestDataBlockEncoders { DataBlockEncoder encoder = encoding.getEncoder(); ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv, getEncodingContext(Compression.Algorithm.NONE, encoding)); - ByteBuffer keyBuffer = encoder.getFirstKeyInBlock(encodedBuffer); + ByteBuffer keyBuffer = encoder.getFirstKeyInBlock(new MultiByteBuffer(encodedBuffer)); KeyValue firstKv = sampleKv.get(0); if (0 != Bytes.compareTo(keyBuffer.array(), keyBuffer.arrayOffset(), keyBuffer.limit(), firstKv.getBuffer(), firstKv.getKeyOffset(), firstKv.getKeyLength())) { @@ -325,6 +328,7 @@ public class TestDataBlockEncoders { seeker.seekToKeyInBlock(keyValue, seekBefore); seeker.rewind(); + // TODO : there is no equals any in PBR.. we may have to add it ByteBuffer actualKeyValue = seeker.getKeyValueBuffer(); ByteBuffer actualKey = seeker.getKeyDeepCopy(); ByteBuffer actualValue = seeker.getValueShallowCopy(); @@ -389,10 +393,10 @@ public class TestDataBlockEncoders { byte[] value0 = new byte[] { 'd' }; byte[] value1 = new byte[] { 0x00 }; if (includesTags) { - kvList.add(new KeyValue(row, family, qualifier0, 0, value0, new Tag[] { new Tag((byte) 1, - "value1") })); - kvList.add(new KeyValue(row, family, qualifier1, 0, value1, new Tag[] { new Tag((byte) 1, - "value1") })); + kvList.add(new KeyValue(row, family, qualifier0, 0, value0, new Tag[] { new ArrayBackedTag( + (byte) 1, "value1") })); + kvList.add(new KeyValue(row, family, qualifier1, 0, value1, new Tag[] { new ArrayBackedTag( + (byte) 1, "value1") })); } else { kvList.add(new KeyValue(row, family, qualifier0, 0, Type.Put, value0)); kvList.add(new KeyValue(row, family, qualifier1, 0, Type.Put, value1)); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java index e087457..9622fa2 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -82,11 +83,13 @@ public class TestEncodedSeekers { public static Collection parameters() { List paramList = new ArrayList(); for (DataBlockEncoding encoding : DataBlockEncoding.values()) { + if(encoding != DataBlockEncoding.PREFIX_TREE){ for (boolean includeTags : new boolean[] { false, true }) { for (boolean compressTags : new boolean[] { false, true }) { paramList.add(new Object[] { encoding, includeTags, compressTags }); } } + } } return paramList; } @@ -148,8 +151,8 @@ public class TestEncodedSeekers { byte[] col = Bytes.toBytes(String.valueOf(j)); byte[] value = dataGenerator.generateRandomSizeValue(key, col); if (includeTags) { - Tag[] tag = new Tag[1]; - tag[0] = new Tag((byte) 1, "Visibility"); + Tag[] tag = new ArrayBackedTag[1]; + tag[0] = new ArrayBackedTag((byte) 1, "Visibility"); KeyValue kv = new KeyValue(key, CF_BYTES, col, HConstants.LATEST_TIMESTAMP, value, tag); put.add(kv); } else { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java index ee664bd..ffd67f7 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java @@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -281,7 +282,7 @@ public class TestPrefixTreeEncoding { kvset.add(kv); } else { KeyValue kv = new KeyValue(getRowKey(batchId, i), CF_BYTES, getQualifier(j), 0l, - getValue(batchId, i, j), new Tag[] { new Tag((byte) 1, "metaValue1") }); + getValue(batchId, i, j), new Tag[] { new ArrayBackedTag((byte) 1, "metaValue1") }); kvset.add(kv); } } @@ -309,7 +310,7 @@ public class TestPrefixTreeEncoding { kvset.add(kv); } else { KeyValue kv = new KeyValue(getRowKey(batchId, i), CF_BYTES, getQualifier(j), 0l, - getValue(batchId, i, j), new Tag[] { new Tag((byte) 1, "metaValue1") }); + getValue(batchId, i, j), new Tag[] { new ArrayBackedTag((byte) 1, "metaValue1") }); kvset.add(kv); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java index 5ef8cf0..1e98415 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.util.ChecksumType; +import org.apache.hadoop.hbase.util.MultiByteBuffer; public class CacheTestUtils { @@ -253,23 +254,17 @@ public class CacheTestUtils { new CacheableDeserializer() { @Override - public Cacheable deserialize(ByteBuffer b) throws IOException { - int len = b.getInt(); - Thread.yield(); - byte buf[] = new byte[len]; - b.get(buf); - return new ByteArrayCacheable(buf); - } - - @Override public int getDeserialiserIdentifier() { return deserializerIdentifier; } @Override - public Cacheable deserialize(ByteBuffer b, boolean reuse) - throws IOException { - return deserialize(b); + public Cacheable deserialize(MultiByteBuffer b, boolean reuse) throws IOException { + int len = b.getInt(); + Thread.yield(); + byte buf[] = new byte[len]; + b.get(buf); + return new ByteArrayCacheable(buf); } }; @@ -326,8 +321,8 @@ public class CacheTestUtils { // declare our data size to be smaller than it by the serialization space // required. - ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize - - HFileBlock.EXTRA_SERIALIZATION_SPACE); + MultiByteBuffer cachedBuffer = new MultiByteBuffer(ByteBuffer.allocate(blockSize + - HFileBlock.EXTRA_SERIALIZATION_SPACE)); rand.nextBytes(cachedBuffer.array()); cachedBuffer.rewind(); int onDiskSizeWithoutHeader = blockSize @@ -348,11 +343,10 @@ public class CacheTestUtils { .withBytesPerCheckSum(0) .withChecksumType(ChecksumType.NULL) .build(); - HFileBlock generated = new HFileBlock(BlockType.DATA, - onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader, - prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER, - blockSize, - onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, meta); + HFileBlock generated = new HFileBlock(BlockType.DATA, onDiskSizeWithoutHeader, + uncompressedSizeWithoutHeader, prevBlockOffset, cachedBuffer, + HFileBlock.DONT_FILL_HEADER, blockSize, onDiskSizeWithoutHeader + + HConstants.HFILEBLOCK_HEADER_SIZE, meta); String strKey; /* No conflicting keys */ diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java index c5fcc3c..eeae553 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.Before; @@ -71,14 +72,7 @@ public class TestCacheConfig { } @Override - public Cacheable deserialize(ByteBuffer b, boolean reuse) throws IOException { - LOG.info("Deserialized " + b + ", reuse=" + reuse); - return cacheable; - } - - @Override - public Cacheable deserialize(ByteBuffer b) throws IOException { - LOG.info("Deserialized " + b); + public Cacheable deserialize(MultiByteBuffer b, boolean reuse) throws IOException { return cacheable; } }; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java index 7ec7e08..c0d57b9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -1,6 +1,6 @@ /* * - * Licensed to the Apache Software Foundation (ASF) under one +\ * 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 @@ -36,6 +36,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -399,7 +400,7 @@ public class TestCacheOnWrite { byte[] value = TestHFileWriterV2.randomValue(rand); KeyValue kv; if(useTags) { - Tag t = new Tag((byte) 1, "visibility"); + Tag t = new ArrayBackedTag((byte) 1, "visibility"); List tagList = new ArrayList(); tagList.add(t); Tag[] tags = new Tag[1]; @@ -453,7 +454,7 @@ public class TestCacheOnWrite { String valueStr = "value_" + rowStr + "_" + qualStr; for (int iTS = 0; iTS < 5; ++iTS) { if (useTags) { - Tag t = new Tag((byte) 1, "visibility"); + Tag t = new ArrayBackedTag((byte) 1, "visibility"); Tag[] tags = new Tag[1]; tags[0] = t; KeyValue kv = new KeyValue(Bytes.toBytes(rowStr), cfBytes, Bytes.toBytes(qualStr), diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java index 80266af..4e18f92 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java @@ -27,7 +27,6 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -37,12 +36,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.ChecksumType; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -121,7 +121,9 @@ public class TestChecksum { assertEquals(algo == GZ ? 2173 : 4936, b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes()); // read data back from the hfile, exclude header and checksum - ByteBuffer bb = b.unpack(meta, hbr).getBufferWithoutHeader(); // read back data + MultiByteBuffer bb = b.unpack(meta, hbr).getBufferWithoutHeader(); // read back data + assert bb.hasArray(); + // TODO DataInputStream in = new DataInputStream( new ByteArrayInputStream( bb.array(), bb.arrayOffset(), bb.limit())); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index 3855629..a928081 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -23,7 +23,6 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile.Reader; @@ -44,9 +44,8 @@ import org.apache.hadoop.hbase.io.hfile.HFile.Writer; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.io.Writable; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -68,18 +67,6 @@ public class TestHFile extends HBaseTestCase { private final int minBlockSize = 512; private static String localFormatter = "%010d"; private static CacheConfig cacheConf = null; - private Map startingMetrics; - - @Before - public void setUp() throws Exception { - super.setUp(); - } - - @After - public void tearDown() throws Exception { - super.tearDown(); - } - /** * Test empty HFile. @@ -168,7 +155,7 @@ public class TestHFile extends HBaseTestCase { for (int i = start; i < (start + n); i++) { String key = String.format(localFormatter, Integer.valueOf(i)); if (useTags) { - Tag t = new Tag((byte) 1, "myTag1"); + Tag t = new ArrayBackedTag((byte) 1, "myTag1"); Tag[] tags = new Tag[1]; tags[0] = t; kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"), @@ -199,10 +186,9 @@ public class TestHFile extends HBaseTestCase { String valStr = value + keyStr; KeyValue kv = new KeyValue(Bytes.toBytes(keyStr), Bytes.toBytes("family"), Bytes.toBytes("qual"), Bytes.toBytes(valStr)); - byte[] keyBytes = new KeyValue.KeyOnlyKeyValue(Bytes.toBytes(key), 0, - Bytes.toBytes(key).length).getKey(); + byte[] keyBytes = Bytes.toBytes(key); assertTrue("bytes for keys do not match " + keyStr + " " + - Bytes.toString(Bytes.toBytes(key)), + Bytes.toString(keyBytes), Arrays.equals(kv.getKey(), keyBytes)); byte [] valBytes = Bytes.toBytes(val); assertTrue("bytes for vals do not match " + valStr + " " + @@ -274,7 +260,7 @@ public class TestHFile extends HBaseTestCase { // read the key and see if it matches ByteBuffer readKey = scanner.getKey(); assertTrue("seeked key does not match", Arrays.equals(getSomeKey(50), - Bytes.toBytes(readKey))); + Bytes.toBytes(readKey))); scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(0))); ByteBuffer val1 = scanner.getValue(); @@ -322,11 +308,11 @@ public class TestHFile extends HBaseTestCase { private void readNumMetablocks(Reader reader, int n) throws IOException { for (int i = 0; i < n; i++) { - ByteBuffer actual = reader.getMetaBlock("HFileMeta" + i, false); + MultiByteBuffer actual = reader.getMetaBlock("HFileMeta" + i, false).getBufferWithoutHeader(); ByteBuffer expected = ByteBuffer.wrap(("something to test" + i).getBytes()); assertEquals("failed to match metadata", - Bytes.toStringBinary(expected), Bytes.toStringBinary(actual)); + Bytes.toStringBinary(expected), MultiByteBuffer.toStringBinary(actual)); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java index eb1f1bb..c97916a 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java @@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ; import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; @@ -50,6 +53,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.compress.Compression; @@ -60,6 +64,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.Compressor; import org.junit.Before; @@ -163,8 +168,8 @@ public class TestHFileBlock { if (!useTag) { keyValues.add(new KeyValue(row, family, qualifier, timestamp, value)); } else { - keyValues.add(new KeyValue(row, family, qualifier, timestamp, value, new Tag[] { new Tag( - (byte) 1, Bytes.toBytes("myTagVal")) })); + keyValues.add(new KeyValue(row, family, qualifier, timestamp, value, + new Tag[] { new ArrayBackedTag((byte) 1, Bytes.toBytes("myTagVal")) })); } } @@ -435,7 +440,8 @@ public class TestHFileBlock { assertTrue("Packed heapSize should be < unpacked heapSize", packedHeapsize < blockUnpacked.heapSize()); } - ByteBuffer actualBuffer = blockUnpacked.getBufferWithoutHeader(); + MultiByteBuffer mbb = blockUnpacked.getBufferWithoutHeader(); + ByteBuffer actualBuffer = mbb.asSubBuffer(mbb.limit()); if (encoding != DataBlockEncoding.NONE) { // We expect a two-byte big-endian encoding id. assertEquals( @@ -459,7 +465,7 @@ public class TestHFileBlock { ByteBuffer serialized = ByteBuffer.allocate(blockFromHFile.getSerializedLength()); blockFromHFile.serialize(serialized); HFileBlock deserialized = - (HFileBlock) blockFromHFile.getDeserializer().deserialize(serialized, reuseBuffer); + (HFileBlock) blockFromHFile.getDeserializer().deserialize(new MultiByteBuffer(serialized), reuseBuffer); assertEquals( "Serialization did not preserve block state. reuseBuffer=" + reuseBuffer, blockFromHFile, deserialized); @@ -593,7 +599,8 @@ public class TestHFileBlock { b = b.unpack(meta, hbr); // b's buffer has header + data + checksum while // expectedContents have header + data only - ByteBuffer bufRead = b.getBufferWithHeader(); + MultiByteBuffer mbb = b.getBufferWithHeader(); + ByteBuffer bufRead = mbb.asSubBuffer(mbb.limit()); ByteBuffer bufExpected = expectedContents.get(i); boolean bytesAreCorrect = Bytes.compareTo(bufRead.array(), bufRead.arrayOffset(), @@ -614,10 +621,11 @@ public class TestHFileBlock { + Bytes.toStringBinary(bufRead.array(), bufRead.arrayOffset(), Math.min(32 + 10, bufRead.limit())); if (detailedLogging) { - LOG.warn("expected header" + + // TODO correct + /*LOG.warn("expected header" + HFileBlock.toStringHeader(bufExpected) + "\nfound header" + - HFileBlock.toStringHeader(bufRead)); + HFileBlock.toStringHeader(bufRead));*/ LOG.warn("bufread offset " + bufRead.arrayOffset() + " limit " + bufRead.limit() + " expected offset " + bufExpected.arrayOffset() + @@ -835,8 +843,7 @@ public class TestHFileBlock { .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM) .withChecksumType(ChecksumType.NULL).build(); HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf, - HFileBlock.FILL_HEADER, -1, - 0, meta); + HFileBlock.FILL_HEADER, -1, 0, meta); long byteBufferExpectedSize = ClassSize.align(ClassSize.estimateBase(buf.getClass(), true) + HConstants.HFILEBLOCK_HEADER_SIZE + size); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java index fc44f3c..0d8c4c0 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java @@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ; import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; @@ -52,6 +55,7 @@ import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.Compressor; import org.junit.Before; @@ -310,7 +314,8 @@ public class TestHFileBlockCompatibility { assertEquals((int) encodedSizes.get(blockId), b.getUncompressedSizeWithoutHeader()); - ByteBuffer actualBuffer = b.getBufferWithoutHeader(); + MultiByteBuffer mbb = b.getBufferWithoutHeader(); + ByteBuffer actualBuffer = mbb.asSubBuffer(mbb.limit()); if (encoding != DataBlockEncoding.NONE) { // We expect a two-byte big-endian encoding id. assertEquals(0, actualBuffer.get(0)); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java index 939c019..ac7f543 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java @@ -52,9 +52,11 @@ import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk; import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -163,8 +165,13 @@ public class TestHFileBlockIndex { public BlockReaderWrapper(HFileBlock.FSReader realReader) { this.realReader = realReader; } + @Override + public void returnBlock(HFileBlock block) { + + } + @Override public HFileBlock readBlock(long offset, long onDiskSize, boolean cacheBlock, boolean pread, boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, @@ -182,9 +189,9 @@ public class TestHFileBlockIndex { prevOffset = offset; prevOnDiskSize = onDiskSize; prevPread = pread; - return prevBlock; } + } private void readIndex(boolean useTags) throws IOException { @@ -389,7 +396,7 @@ public class TestHFileBlockIndex { dos.writeInt(curAllEntriesSize); - ByteBuffer nonRootIndex = ByteBuffer.wrap(baos.toByteArray()); + MultiByteBuffer nonRootIndex = new MultiByteBuffer(ByteBuffer.wrap(baos.toByteArray())); for (int i = 0; i < numTotalKeys; ++i) { byte[] searchKey = keys.get(i); byte[] arrayHoldingKey = new byte[searchKey.length + @@ -600,7 +607,7 @@ public class TestHFileBlockIndex { while ((block = iter.nextBlock()) != null) { if (block.getBlockType() != BlockType.LEAF_INDEX) return; - ByteBuffer b = block.getBufferReadOnly(); + MultiByteBuffer b = block.getBufferReadOnly(); int n = b.getInt(); // One int for the number of items, and n + 1 for the secondary index. int entriesOffset = Bytes.SIZEOF_INT * (n + 2); @@ -645,9 +652,10 @@ public class TestHFileBlockIndex { private void assertArrayEqualsBuffer(String msgPrefix, byte[] arr, ByteBuffer buf) { - assertEquals(msgPrefix + ": expected " + Bytes.toStringBinary(arr) - + ", actual " + Bytes.toStringBinary(buf), 0, Bytes.compareTo(arr, 0, - arr.length, buf.array(), buf.arrayOffset(), buf.limit())); + assertEquals( + msgPrefix + ": expected " + Bytes.toStringBinary(arr) + ", actual " + + Bytes.toStringBinary(buf), 0, + ByteBufferUtils.compareTo(buf, 0, buf.limit(), arr, 0, arr.length)); } /** Check a key/value pair after it was read by the reader */ @@ -660,7 +668,5 @@ public class TestHFileBlockIndex { assertArrayEqualsBuffer(msgPrefix + "Invalid value", expectedValue, valueRead); } - - } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java index 3cdc92b..6f6ba5f 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java @@ -124,9 +124,8 @@ public class TestHFileDataBlockEncoder { .withBlockSize(0) .withChecksumType(ChecksumType.NULL) .build(); - HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf, - HFileBlock.FILL_HEADER, 0, - 0, hfileContext); + HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf, HFileBlock.FILL_HEADER, + 0, 0, hfileContext); HFileBlock cacheBlock = createBlockOnDisk(kvs, block, useTags); assertEquals(headerSize, cacheBlock.getDummyHeaderForVersion().length); } @@ -172,9 +171,8 @@ public class TestHFileDataBlockEncoder { .withBlockSize(0) .withChecksumType(ChecksumType.NULL) .build(); - HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf, - HFileBlock.FILL_HEADER, 0, - 0, meta); + HFileBlock b = new HFileBlock(BlockType.DATA, size, size, -1, buf, HFileBlock.FILL_HEADER, 0, + 0, meta); return b; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java index 76a8200..d9bc204 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java @@ -135,7 +135,7 @@ public class TestHFileSeek extends TestCase { Writer writer = HFile.getWriterFactoryNoCache(conf) .withOutputStream(fout) .withFileContext(context) - .withComparator(new KeyValue.RawBytesComparator()) + .withComparator(new KeyValue.KVComparator()) .create(); try { BytesWritable key = new BytesWritable(); @@ -195,7 +195,7 @@ public class TestHFileSeek extends TestCase { kSampler.next(key); byte [] k = new byte [key.getLength()]; System.arraycopy(key.getBytes(), 0, k, 0, key.getLength()); - if (scanner.seekTo(KeyValue.createKeyValueFromKey(k)) >= 0) { + if (scanner.seekTo(new KeyValue.KeyOnlyKeyValue(k, 0, k.length)) >= 0) { ByteBuffer bbkey = scanner.getKey(); ByteBuffer bbval = scanner.getValue(); totalBytes += bbkey.limit(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java index 42e918a..89b5019 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java @@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.io.hfile; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.ByteArrayInputStream; @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; @@ -197,7 +198,7 @@ public class TestHFileWriterV2 { assertFalse(block.isUnpacked()); block = block.unpack(meta, blockReader); } - ByteBuffer buf = block.getBufferWithoutHeader(); + MultiByteBuffer buf = block.getBufferWithoutHeader(); while (buf.hasRemaining()) { int keyLen = buf.getInt(); int valueLen = buf.getInt(); @@ -242,7 +243,8 @@ public class TestHFileWriterV2 { .unpack(meta, blockReader); assertEquals(BlockType.META, block.getBlockType()); Text t = new Text(); - ByteBuffer buf = block.getBufferWithoutHeader(); + MultiByteBuffer mbb = block.getBufferWithoutHeader(); + ByteBuffer buf = mbb.asSubBuffer(mbb.limit()); if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) { throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName()); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java index f96e8ef..0832723 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.KVComparator; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; @@ -141,7 +143,7 @@ public class TestHFileWriterV3 { for (int j = 0; j < 1 + rand.nextInt(4); j++) { byte[] tagBytes = new byte[16]; rand.nextBytes(tagBytes); - tags.add(new Tag((byte) 1, tagBytes)); + tags.add(new ArrayBackedTag((byte) 1, tagBytes)); } keyValue = new KeyValue(keyBytes, null, null, HConstants.LATEST_TIMESTAMP, valueBytes, tags); @@ -223,7 +225,7 @@ public class TestHFileWriterV3 { HFileBlock block = blockReader.readBlockData(curBlockPos, -1, -1, false) .unpack(context, blockReader); assertEquals(BlockType.DATA, block.getBlockType()); - ByteBuffer buf = block.getBufferWithoutHeader(); + MultiByteBuffer buf = block.getBufferWithoutHeader(); int keyLen = -1; while (buf.hasRemaining()) { @@ -283,7 +285,8 @@ public class TestHFileWriterV3 { .unpack(context, blockReader); assertEquals(BlockType.META, block.getBlockType()); Text t = new Text(); - ByteBuffer buf = block.getBufferWithoutHeader(); + MultiByteBuffer mbb = block.getBufferWithoutHeader(); + ByteBuffer buf = mbb.asSubBuffer(mbb.limit()); if (Writables.getWritable(buf.array(), buf.arrayOffset(), buf.limit(), t) == null) { throw new IOException("Failed to deserialize block " + this + " into a " + t.getClass().getSimpleName()); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java index 2fd3684..0e0f543 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java @@ -17,7 +17,17 @@ */ package org.apache.hadoop.hbase.io.hfile; -import com.google.common.collect.Iterables; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Random; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -39,14 +49,7 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Random; - -import static org.junit.Assert.*; +import com.google.common.collect.Iterables; /** * A kind of integration test at the intersection of {@link HFileBlock}, {@link CacheConfig}, diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java index 4ceafb4..7fbecdc 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java @@ -19,7 +19,7 @@ */ package org.apache.hadoop.hbase.io.hfile; -import static org.junit.Assert.*; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Random; @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.regionserver.StoreFile; - import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java index 3a0fdf7..316cde7 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -83,7 +84,7 @@ public class TestReseekTo { Bytes.toBytes(value)); writer.append(kv); } else if (tagUsage == TagUsage.ONLY_TAG) { - Tag t = new Tag((byte) 1, "myTag1"); + Tag t = new ArrayBackedTag((byte) 1, "myTag1"); Tag[] tags = new Tag[1]; tags[0] = t; kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"), @@ -91,7 +92,7 @@ public class TestReseekTo { writer.append(kv); } else { if (key % 4 == 0) { - Tag t = new Tag((byte) 1, "myTag1"); + Tag t = new ArrayBackedTag((byte) 1, "myTag1"); Tag[] tags = new Tag[1]; tags[0] = t; kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"), diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java index b9a126f..37d6020 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -47,7 +48,7 @@ public class TestSeekTo extends HBaseTestCase { return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"), Bytes.toBytes("value")); } else if (tagUsage == TagUsage.ONLY_TAG) { - Tag t = new Tag((byte) 1, "myTag1"); + Tag t = new ArrayBackedTag((byte) 1, "myTag1"); Tag[] tags = new Tag[1]; tags[0] = t; return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"), @@ -59,7 +60,7 @@ public class TestSeekTo extends HBaseTestCase { Bytes.toBytes("qualifier"), Bytes.toBytes("value")); } else { switchKVs = false; - Tag t = new Tag((byte) 1, "myTag1"); + Tag t = new ArrayBackedTag((byte) 1, "myTag1"); Tag[] tags = new Tag[1]; tags[0] = t; return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java index d29be01..26a59c6 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java @@ -27,13 +27,13 @@ import java.util.Arrays; import java.util.List; import java.util.Random; -import org.apache.hadoop.hbase.testclassification.IOTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; import org.apache.hadoop.hbase.io.hfile.CacheTestUtils; import org.apache.hadoop.hbase.io.hfile.Cacheable; import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.BucketSizeInfo; import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.IndexStatistics; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; import org.junit.Before; import org.junit.Test; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java index 511f942..f528041 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.MultiByteBuffer; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -73,4 +74,43 @@ public class TestByteBufferIOEngine { assert testOffsetAtStartNum == 0; assert testOffsetAtEndNum == 0; } + + @Test + public void testByteBufferIOEngineWithMBB() throws Exception { + int capacity = 32 * 1024 * 1024; // 32 MB + int testNum = 100; + int maxBlockSize = 64 * 1024; + ByteBufferIOEngine ioEngine = new ByteBufferIOEngine(capacity, false); + int testOffsetAtStartNum = testNum / 10; + int testOffsetAtEndNum = testNum / 10; + for (int i = 0; i < testNum; i++) { + byte val = (byte) (Math.random() * 255); + int blockSize = (int) (Math.random() * maxBlockSize); + if (blockSize == 0) { + blockSize = 1; + } + byte[] byteArray = new byte[blockSize]; + for (int j = 0; j < byteArray.length; ++j) { + byteArray[j] = val; + } + ByteBuffer srcBuffer = ByteBuffer.wrap(byteArray); + int offset = 0; + if (testOffsetAtStartNum > 0) { + testOffsetAtStartNum--; + offset = 0; + } else if (testOffsetAtEndNum > 0) { + testOffsetAtEndNum--; + offset = capacity - blockSize; + } else { + offset = (int) (Math.random() * (capacity - maxBlockSize)); + } + ioEngine.write(srcBuffer, offset); + MultiByteBuffer read = ioEngine.read(offset, blockSize); + for (int j = 0; j < byteArray.length; ++j) { + assertTrue(srcBuffer.get(j) == read.get(j)); + } + } + assert testOffsetAtStartNum == 0; + assert testOffsetAtEndNum == 0; + } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java index 43e29ad..b4885b0 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.Assert; @@ -125,8 +126,8 @@ public abstract class TableSnapshotInputFormatTestBase { Cell cell = scanner.current(); //assert that all Cells in the Result have the same key - Assert.assertEquals(0, Bytes.compareTo(row, 0, row.length, - cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())); + Assert.assertEquals(0, ByteBufferUtils.compareTo(row, 0, row.length, + cell.getRowBuffer(), cell.getRowOffset(), cell.getRowLength())); } for (int j = 0; j < FAMILIES.length; j++) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index dc142d6..70c86ab 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; @@ -53,6 +51,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2; import org.apache.hadoop.hbase.wal.DefaultWALProvider; import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index 200790b..9ddbbb6 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -533,10 +533,17 @@ public class TestDefaultMemStore extends TestCase { int rowId = startRowId + i; Cell left = results.get(0); byte[] row1 = Bytes.toBytes(rowId); - assertTrue( - "Row name", - KeyValue.COMPARATOR.compareRows(left.getRowArray(), left.getRowOffset(), - (int) left.getRowLength(), row1, 0, row1.length) == 0); + if(left.hasArray()) { + assertTrue( + "Row name", + KeyValue.COMPARATOR.compareRows(left.getRowArray(), left.getRowOffset(), + (int) left.getRowLength(), row1, 0, row1.length) == 0); + } else { + assertTrue( + "Row name", + KeyValue.COMPARATOR.compareRows(left.getRowBuffer(), left.getRowOffset(), + (int) left.getRowLength(), row1, 0, row1.length) == 0); + } assertEquals("Count of columns", QUALIFIER_COUNT, results.size()); List row = new ArrayList(); for (Cell kv : results) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index ca5135d..e8ff871 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -70,6 +70,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; @@ -142,6 +143,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; @@ -3898,6 +3900,7 @@ public class TestHRegion { for (int i = 0; i < testCount; i++) { LOG.info("testWritesWhileGetting verify turn " + i); boolean previousEmpty = result == null || result.isEmpty(); + LOG.info("The get is "+get); result = region.get(get); if (!result.isEmpty() || !previousEmpty || i > compactInterval) { assertEquals("i=" + i, expectedCount, result.size()); @@ -4395,7 +4398,7 @@ public class TestHRegion { // we just got the latest version assertEquals(kvs.size(), 1); Cell kv = kvs.get(0); - assertEquals(expected, Bytes.toLong(kv.getValueArray(), kv.getValueOffset())); + assertEquals(expected, ByteBufferUtils.getLong(kv.getValueBuffer(), kv.getValueOffset())); this.region = null; } @@ -4487,7 +4490,13 @@ public class TestHRegion { assertEquals(kvs.size(), 1); Cell kv = kvs.get(0); byte[] appendResult = new byte[kv.getValueLength()]; - System.arraycopy(kv.getValueArray(), kv.getValueOffset(), appendResult, 0, kv.getValueLength()); + if (kv.hasArray()) { + System.arraycopy(kv.getValueArray(), kv.getValueOffset(), appendResult, 0, + kv.getValueLength()); + } else { + ByteBufferUtils.copyFromBufferToByteArray(appendResult, kv.getValueBuffer(), + kv.getValueOffset(), 0, kv.getValueLength()); + } assertArrayEquals(expected, appendResult); this.region = null; } @@ -6051,14 +6060,14 @@ public class TestHRegion { region.put(new Put(row).add(new KeyValue(row, fam1, q1, now, HConstants.EMPTY_BYTE_ARRAY, new Tag[] { // TTL tags specify ts in milliseconds - new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } ))); + new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } ))); // Add a cell that will expire after 10 seconds via family setting region.put(new Put(row).add(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY)); // Add a cell that will expire in 15 seconds via cell TTL region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY, new Tag[] { // TTL tags specify ts in milliseconds - new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } ))); + new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } ))); // Add a cell that will expire in 20 seconds via family setting region.put(new Put(row).add(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY)); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java index b96a6a5..a7de2e3 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java @@ -34,11 +34,13 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.io.hfile.Cacheable; import org.apache.hadoop.hbase.io.hfile.CachedBlock; +import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache; import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext; @@ -400,6 +402,12 @@ public class TestHeapMemoryManager { public BlockCache[] getBlockCaches() { return null; } + + @Override + public boolean returnBlock(BlockCacheKey cacheKey, HFileBlock block) { + // TODO Auto-generated method stub + return true; + } } private static class MemstoreFlusherStub implements FlushRequester { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java index 80333e8..f82af83 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.nio.ByteBuffer; import java.util.List; import java.util.Random; diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java index 170bdd4..8e2603d 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.*; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.Random; @@ -126,14 +127,14 @@ public class TestMemStoreLAB { if (rec.size == 0) continue; Map mapForThisByteArray = - mapsByChunk.get(rec.alloc.getBytes()); - if (mapForThisByteArray == null) { - mapForThisByteArray = Maps.newTreeMap(); - mapsByChunk.put(rec.alloc.getBytes(), mapForThisByteArray); - } - AllocRecord oldVal = mapForThisByteArray.put(rec.alloc.getOffset(), rec); - assertNull("Already had an entry " + oldVal + " for allocation " + rec, - oldVal); + mapsByChunk.get(rec.alloc.getBytes()); + if (mapForThisByteArray == null) { + mapForThisByteArray = Maps.newTreeMap(); + mapsByChunk.put(rec.alloc.getBytes(), mapForThisByteArray); + } + AllocRecord oldVal = mapForThisByteArray.put(rec.alloc.getOffset(), rec); + assertNull("Already had an entry " + oldVal + " for allocation " + rec, + oldVal); } assertEquals("Sanity check test", sizeCounted, totalAllocated.get()); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java index 2df2f5a..7dbca2e 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java @@ -114,7 +114,6 @@ public class TestQueryMatcher extends HBaseTestCase { List actual = new ArrayList(); KeyValue k = memstore.get(0); qm.setRow(k.getRowArray(), k.getRowOffset(), k.getRowLength()); - for (KeyValue kv : memstore){ actual.add(qm.match(kv)); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java index 1423127..c678299 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -191,6 +191,7 @@ public class TestScanner { while (hasMore) { hasMore = NextState.hasMoreValues(s.next(results)); for (Cell kv : results) { + System.out.println(kv); assertTrue(Bytes.compareTo(CellUtil.cloneRow(kv), stopRow) <= 0); } results.clear(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index e5a5022..9a2bced 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -55,11 +55,10 @@ import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; import org.apache.hadoop.hbase.util.FSUtils; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -81,16 +80,6 @@ public class TestStoreFile extends HBaseTestCase { private static final int CKBYTES = 512; private static String TEST_FAMILY = "cf"; - @Before - public void setUp() throws Exception { - super.setUp(); - } - - @After - public void tearDown() throws Exception { - super.tearDown(); - } - /** * Write a file and then assert that we can read from top and bottom halves * using two HalfMapFiles. @@ -184,7 +173,8 @@ public class TestStoreFile extends HBaseTestCase { HFileScanner s = refHsf.createReader().getScanner(false, false); for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) { ByteBuffer bb = s.getKey(); - kv = KeyValue.createKeyValueFromKey(bb); + byte[] key = Bytes.toBytes(bb); + kv = KeyValue.createKeyValueFromKey(key, 0, key.length); if (first) { assertTrue(Bytes.equals(kv.getRow(), midRow)); first = false; @@ -330,48 +320,48 @@ public class TestStoreFile extends HBaseTestCase { this.fs, topPath, conf, cacheConf, BloomType.NONE).createReader(); StoreFile.Reader bottom = new StoreFile( this.fs, bottomPath, conf, cacheConf, BloomType.NONE).createReader(); - ByteBuffer previous = null; + byte[] previous = null; LOG.info("Midkey: " + midKV.toString()); - ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey); try { // Now make two HalfMapFiles and assert they can read the full backing // file, one from the top and the other from the bottom. // Test bottom half first. // Now test reading from the top. boolean first = true; - ByteBuffer key = null; + ByteBuffer bb = null; + byte[] key = null; HFileScanner topScanner = top.getScanner(false, false); while ((!topScanner.isSeeked() && topScanner.seekTo()) || (topScanner.isSeeked() && topScanner.next())) { - key = topScanner.getKey(); + bb = topScanner.getKey(); + key = Bytes.toBytes(bb); - if (topScanner.getReader().getComparator().compareFlatKey(key.array(), - key.arrayOffset(), key.limit(), midkey, 0, midkey.length) < 0) { + if (topScanner.getReader().getComparator().compareFlatKey(key, + 0, key.length, midkey, 0, midkey.length) < 0) { fail("key=" + Bytes.toStringBinary(key) + " < midkey=" + Bytes.toStringBinary(midkey)); } if (first) { first = false; - LOG.info("First in top: " + Bytes.toString(Bytes.toBytes(key))); + LOG.info("First in top: " + Bytes.toString(key)); } } - LOG.info("Last in top: " + Bytes.toString(Bytes.toBytes(key))); + LOG.info("Last in top: " + Bytes.toString(key)); first = true; HFileScanner bottomScanner = bottom.getScanner(false, false); while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) || bottomScanner.next()) { - previous = bottomScanner.getKey(); - key = bottomScanner.getKey(); + previous = Bytes.toBytes(bottomScanner.getKey()); + key = Bytes.toBytes(bottomScanner.getKey()); if (first) { first = false; - LOG.info("First in bottom: " + - Bytes.toString(Bytes.toBytes(previous))); + LOG.info("First in bottom: " + Bytes.toString(previous)); } - assertTrue(key.compareTo(bbMidkeyBytes) < 0); + assertTrue(Bytes.compareTo(key, midkey) < 0); } if (previous != null) { - LOG.info("Last in bottom: " + Bytes.toString(Bytes.toBytes(previous))); + LOG.info("Last in bottom: " + Bytes.toString(previous)); } // Remove references. regionFs.cleanupDaughterRegion(topHri); @@ -393,9 +383,9 @@ public class TestStoreFile extends HBaseTestCase { topScanner = top.getScanner(false, false); while ((!topScanner.isSeeked() && topScanner.seekTo()) || topScanner.next()) { - key = topScanner.getKey(); - assertTrue(topScanner.getReader().getComparator().compareFlatKey(key.array(), - key.arrayOffset(), key.limit(), badmidkey, 0, badmidkey.length) >= 0); + key = Bytes.toBytes(topScanner.getKey()); + assertTrue(topScanner.getReader().getComparator().compareFlatKey(key, + 0, key.length, badmidkey, 0, badmidkey.length) >= 0); if (first) { first = false; KeyValue keyKV = KeyValue.createKeyValueFromKey(key); @@ -427,7 +417,7 @@ public class TestStoreFile extends HBaseTestCase { bottomScanner = bottom.getScanner(false, false); while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) || bottomScanner.next()) { - key = bottomScanner.getKey(); + key = Bytes.toBytes(bottomScanner.getKey()); if (first) { first = false; keyKV = KeyValue.createKeyValueFromKey(key); @@ -563,8 +553,9 @@ public class TestStoreFile extends HBaseTestCase { for (int i = 0; i < 2000; i++) { String row = String.format(localFormatter, i); byte[] rowKey = Bytes.toBytes(row); - boolean exists = reader.passesDeleteFamilyBloomFilter(rowKey, 0, - rowKey.length); + KeyValue kv = new KeyValue(row.getBytes(), "family".getBytes(), + "col".getBytes()); + boolean exists = reader.passesDeleteFamilyBloomFilter(kv); if (i % 2 == 0) { if (!exists) falseNeg++; @@ -909,9 +900,13 @@ public class TestStoreFile extends HBaseTestCase { assertTrue(Bytes.compareTo( keyv1.getBuffer(), keyv1.getKeyOffset(), keyv1.getKeyLength(), keyv2.getBuffer(), keyv2.getKeyOffset(), keyv2.getKeyLength()) == 0); - assertTrue(Bytes.compareTo( - kv1.getValueArray(), kv1.getValueOffset(), kv1.getValueLength(), - kv2.getValueArray(), kv2.getValueOffset(), kv2.getValueLength()) == 0); + if (kv1.hasArray()) { + assertTrue(Bytes.compareTo(kv1.getValueArray(), kv1.getValueOffset(), kv1.getValueLength(), + kv2.getValueArray(), kv2.getValueOffset(), kv2.getValueLength()) == 0); + } else { + assertTrue(ByteBufferUtils.compareTo(kv1.getValueBuffer(), kv1.getValueOffset(), kv1.getValueLength(), + kv2.getValueBuffer(), kv2.getValueOffset(), kv2.getValueLength()) == 0); + } } assertNull(scannerTwo.next()); assertEquals(startHit + 6, cs.getHitCount()); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java index 1bcb7c9..8b5c290 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java @@ -28,12 +28,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileContext; @@ -82,11 +85,10 @@ public class TestStoreFileScannerWithTagCompression { kv = s.next(); kv = s.next(); byte[] key5 = Bytes.toBytes("k5"); - assertTrue(Bytes.equals(key5, 0, key5.length, kv.getRowArray(), kv.getRowOffset(), - kv.getRowLength())); + assertTrue(CellUtil.matchingRow(kv, key5)); List tags = KeyValueUtil.ensureKeyValue(kv).getTags(); assertEquals(1, tags.size()); - assertEquals("tag3", Bytes.toString(tags.get(0).getValue())); + assertEquals("tag3", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); } finally { s.close(); } @@ -97,9 +99,9 @@ public class TestStoreFileScannerWithTagCompression { byte[] qualifier = Bytes.toBytes("q"); long now = System.currentTimeMillis(); byte[] b = Bytes.toBytes("k1"); - Tag t1 = new Tag((byte) 1, "tag1"); - Tag t2 = new Tag((byte) 2, "tag2"); - Tag t3 = new Tag((byte) 3, "tag3"); + Tag t1 = new ArrayBackedTag((byte) 1, "tag1"); + Tag t2 = new ArrayBackedTag((byte) 2, "tag2"); + Tag t3 = new ArrayBackedTag((byte) 3, "tag3"); try { writer.append(new KeyValue(b, fam, qualifier, now, b, new Tag[] { t1 })); b = Bytes.toBytes("k3"); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java index 14c6ca9..ee99993 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; @@ -41,7 +43,6 @@ import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; @@ -404,7 +405,7 @@ public class TestTags { List tags = TestCoprocessorForTags.tags; assertEquals(3L, Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); assertEquals(1, tags.size()); - assertEquals("tag1", Bytes.toString(tags.get(0).getValue())); + assertEquals("tag1", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); TestCoprocessorForTags.checkTagPresence = false; TestCoprocessorForTags.tags = null; @@ -422,7 +423,7 @@ public class TestTags { // We cannot assume the ordering of tags List tagValues = new ArrayList(); for (Tag tag: tags) { - tagValues.add(Bytes.toString(tag.getValue())); + tagValues.add(Bytes.toString(TagUtil.getTagValue(tag))); } assertTrue(tagValues.contains("tag1")); assertTrue(tagValues.contains("tag2")); @@ -446,7 +447,7 @@ public class TestTags { tags = TestCoprocessorForTags.tags; assertEquals(4L, Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); assertEquals(1, tags.size()); - assertEquals("tag2", Bytes.toString(tags.get(0).getValue())); + assertEquals("tag2", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); TestCoprocessorForTags.checkTagPresence = false; TestCoprocessorForTags.tags = null; @@ -467,7 +468,7 @@ public class TestTags { kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q)); tags = TestCoprocessorForTags.tags; assertEquals(1, tags.size()); - assertEquals("tag1", Bytes.toString(tags.get(0).getValue())); + assertEquals("tag1", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); TestCoprocessorForTags.checkTagPresence = false; TestCoprocessorForTags.tags = null; @@ -484,7 +485,7 @@ public class TestTags { // We cannot assume the ordering of tags tagValues.clear(); for (Tag tag: tags) { - tagValues.add(Bytes.toString(tag.getValue())); + tagValues.add(Bytes.toString(TagUtil.getTagValue(tag))); } assertTrue(tagValues.contains("tag1")); assertTrue(tagValues.contains("tag2")); @@ -507,7 +508,7 @@ public class TestTags { kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q)); tags = TestCoprocessorForTags.tags; assertEquals(1, tags.size()); - assertEquals("tag2", Bytes.toString(tags.get(0).getValue())); + assertEquals("tag2", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); } finally { TestCoprocessorForTags.checkTagPresence = false; TestCoprocessorForTags.tags = null; @@ -570,7 +571,7 @@ public class TestTags { if (cf == null) { cf = kv.getFamily(); } - Tag tag = new Tag((byte) 1, attribute); + Tag tag = new ArrayBackedTag((byte) 1, attribute); List tagList = new ArrayList(); tagList.add(tag); @@ -611,7 +612,7 @@ public class TestTags { CellScanner cellScanner = result.cellScanner(); if (cellScanner.advance()) { Cell cell = cellScanner.current(); - tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), + tags = TagUtil.asList(cell.getTagsBuffer(), cell.getTagsOffset(), cell.getTagsLength()); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java index 0450904..104f897 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java @@ -24,9 +24,10 @@ import java.util.List; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.io.util.LRUDictionary; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.DataOutputBuffer; @@ -108,7 +109,7 @@ public class TestKeyValueCompression { byte[] value = Bytes.toBytes("myValue"); List tags = new ArrayList(noOfTags); for (int i = 1; i <= noOfTags; i++) { - tags.add(new Tag((byte) i, Bytes.toBytes("tagValue" + i))); + tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i))); } return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java index 501fdda..b02c3fc 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java @@ -29,7 +29,9 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.codec.Codec.Decoder; import org.apache.hadoop.hbase.codec.Codec.Encoder; import org.apache.hadoop.hbase.io.util.LRUDictionary; @@ -69,7 +71,7 @@ public class TestWALCellCodecWithCompression { KeyValue kv = (KeyValue) decoder.current(); List tags = kv.getTags(); assertEquals(1, tags.size()); - assertEquals("tagValue1", Bytes.toString(tags.get(0).getValue())); + assertEquals("tagValue1", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); decoder.advance(); kv = (KeyValue) decoder.current(); tags = kv.getTags(); @@ -78,8 +80,8 @@ public class TestWALCellCodecWithCompression { kv = (KeyValue) decoder.current(); tags = kv.getTags(); assertEquals(2, tags.size()); - assertEquals("tagValue1", Bytes.toString(tags.get(0).getValue())); - assertEquals("tagValue2", Bytes.toString(tags.get(1).getValue())); + assertEquals("tagValue1", Bytes.toString(TagUtil.getTagValue(tags.get(0)))); + assertEquals("tagValue2", Bytes.toString(TagUtil.getTagValue(tags.get(1)))); } private KeyValue createKV(int noOfTags) { @@ -89,7 +91,7 @@ public class TestWALCellCodecWithCompression { byte[] value = Bytes.toBytes("myValue"); List tags = new ArrayList(noOfTags); for (int i = 1; i <= noOfTags; i++) { - tags.add(new Tag((byte) i, Bytes.toBytes("tagValue" + i))); + tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i))); } return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index f1e956c..ca49a95 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -28,6 +28,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -38,13 +39,12 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HBaseAdmin; -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.Table; @@ -75,9 +75,6 @@ public class TestReplicationWithTags { private static ReplicationAdmin replicationAdmin; - private static Connection connection1; - private static Connection connection2; - private static Table htable1; private static Table htable2; @@ -210,7 +207,7 @@ public class TestReplicationWithTags { if (cf == null) { cf = kv.getFamily(); } - Tag tag = new Tag(TAG_TYPE, attribute); + Tag tag = new ArrayBackedTag(TAG_TYPE, attribute); List tagList = new ArrayList(); tagList.add(tag); @@ -238,7 +235,7 @@ public class TestReplicationWithTags { // Check tag presence in the 1st cell in 1st Result if (!results.isEmpty()) { Cell cell = results.get(0); - tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()); + tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()); } } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index b87e7ef..20c9479 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -134,7 +134,7 @@ public class TestReplicationSink { for(int i = 0; i < BATCH_SIZE; i++) { entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells)); } - SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator())); + SINK.replicateEntries(entries, CellUtil.createCellScanner(cells)); Scan scan = new Scan(); ResultScanner scanRes = table1.getScanner(scan); assertEquals(BATCH_SIZE, scanRes.next(BATCH_SIZE).length); @@ -160,7 +160,7 @@ public class TestReplicationSink { i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn, cells)); } - SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator())); + SINK.replicateEntries(entries, CellUtil.createCellScanner(cells)); Scan scan = new Scan(); ResultScanner scanRes = table1.getScanner(scan); assertEquals(BATCH_SIZE/2, scanRes.next(BATCH_SIZE).length); @@ -179,7 +179,7 @@ public class TestReplicationSink { i, KeyValue.Type.Put, cells)); } - SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator())); + SINK.replicateEntries(entries, CellUtil.createCellScanner(cells)); Scan scan = new Scan(); ResultScanner scanRes = table2.getScanner(scan); for(Result res : scanRes) { @@ -198,14 +198,14 @@ public class TestReplicationSink { for(int i = 0; i < 3; i++) { entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells)); } - SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator())); + SINK.replicateEntries(entries, CellUtil.createCellScanner(cells)); entries = new ArrayList(3); cells = new ArrayList(); entries.add(createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn, cells)); entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells)); entries.add(createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn, cells)); - SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator())); + SINK.replicateEntries(entries, CellUtil.createCellScanner(cells)); Scan scan = new Scan(); ResultScanner scanRes = table1.getScanner(scan); @@ -228,7 +228,7 @@ public class TestReplicationSink { for(int i = 3; i < 5; i++) { entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells)); } - SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator())); + SINK.replicateEntries(entries, CellUtil.createCellScanner(cells)); Get get = new Get(Bytes.toBytes(1)); Result res = table1.get(get); assertEquals(0, res.size()); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 9a45116..311d9e1 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; @@ -2415,7 +2416,7 @@ public class TestAccessController extends SecureTestUtil { try { KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY, - new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE, + new Tag[] { new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, ProtobufUtil.toUsersAndPermissions(USER_OWNER.getShortName(), new Permission(Permission.Action.READ)).toByteArray()) }); t.put(new Put(TEST_ROW).add(kv)); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java index e239647..babb955 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java @@ -149,6 +149,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { @Test public void testCellPermissionwithVersions() throws Exception { + final Connection con = ConnectionFactory.createConnection(conf); + final Admin admin = con.getAdmin(); // store two sets of values, one store with a cell level ACL, and one // without verifyAllowed(new AccessTestAction() { @@ -175,6 +177,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO); p.setACL(USER_OTHER.getShortName(), new Permission(Permission.Action.WRITE)); t.put(p); + admin.flush(TEST_TABLE.getTableName()); } finally { t.close(); connection.close(); @@ -921,4 +924,4 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { } assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName()).size()); } -} \ No newline at end of file +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java index b7cbc52..8d1984c 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java @@ -164,6 +164,8 @@ public class TestCellACLs extends SecureTestUtil { .add(TEST_FAMILY, TEST_Q3, ZERO) .add(TEST_FAMILY, TEST_Q4, ZERO); t.put(p); + Admin admin = TEST_UTIL.getHBaseAdmin(); + admin.flush(TEST_TABLE.getTableName()); } finally { t.close(); connection.close(); @@ -440,6 +442,8 @@ public class TestCellACLs extends SecureTestUtil { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO); t.put(p); + Admin admin = TEST_UTIL.getHBaseAdmin(); + admin.flush(TEST_TABLE.getTableName()); } finally { t.close(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java index 3c71a8c..2e8cc15 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java @@ -35,6 +35,7 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode; import org.apache.hadoop.hbase.security.visibility.expression.Operator; +import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import com.google.common.collect.Lists; @@ -74,7 +76,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer private static final byte[] DUMMY_VALUE = new byte[0]; private static final byte STRING_SERIALIZATION_FORMAT = 2; - private static final Tag STRING_SERIALIZATION_FORMAT_TAG = new Tag( + private static final Tag STRING_SERIALIZATION_FORMAT_TAG = new ArrayBackedTag( TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE, new byte[] { STRING_SERIALIZATION_FORMAT }); private final ExpressionParser expressionParser = new ExpressionParser(); @@ -176,8 +178,8 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer } if (cells != null) { for (Cell cell : cells) { - String auth = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(), - cell.getQualifierLength()); + String auth = ByteBufferUtils.toString(cell.getQualifierBuffer(), cell.getQualifierOffset(), + cell.getQualifierLength()); auths.add(auth); } } @@ -284,8 +286,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer boolean visibilityTagPresent = false; // Save an object allocation where we can if (cell.getTagsLength() > 0) { - Iterator tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), - cell.getTagsLength()); + Iterator tagsItr = CellUtil.tagsIterator(cell); while (tagsItr.hasNext()) { boolean includeKV = true; Tag tag = tagsItr.next(); @@ -294,18 +295,18 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer int offset = tag.getTagOffset(); int endOffset = offset + tag.getTagLength(); while (offset < endOffset) { - short len = Bytes.toShort(tag.getBuffer(), offset); + short len = ByteBufferUtils.getShort(tag.getBuffer(), offset); offset += 2; if (len < 0) { // This is a NOT label. len = (short) (-1 * len); - String label = Bytes.toString(tag.getBuffer(), offset, len); + String label = ByteBufferUtils.toString(tag.getBuffer(), offset, len); if (authLabelsFinal.contains(label)) { includeKV = false; break; } } else { - String label = Bytes.toString(tag.getBuffer(), offset, len); + String label = ByteBufferUtils.toString(tag.getBuffer(), offset, len); if (!authLabelsFinal.contains(label)) { includeKV = false; break; @@ -356,7 +357,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer dos.writeShort(bLabel.length); dos.write(bLabel); } - return new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()); + return new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()); } private void extractLabels(ExpressionNode node, List labels, List notLabels) { @@ -467,7 +468,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer for (Tag tag : deleteVisTags) { matchFound = false; for (Tag givenTag : putVisTags) { - if (Bytes.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(), + if (ByteBufferUtils.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(), givenTag.getBuffer(), givenTag.getTagOffset(), givenTag.getTagLength())) { matchFound = true; break; @@ -507,11 +508,11 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer int endOffset = offset + tag.getTagLength(); boolean expressionStart = true; while (offset < endOffset) { - short len = Bytes.toShort(tag.getBuffer(), offset); + short len = ByteBufferUtils.getShort(tag.getBuffer(), offset); offset += 2; if (len < 0) { len = (short) (-1 * len); - String label = Bytes.toString(tag.getBuffer(), offset, len); + String label = ByteBufferUtils.toString(tag.getBuffer(), offset, len); if (expressionStart) { visibilityString.append(VisibilityConstants.OPEN_PARAN + VisibilityConstants.NOT_OPERATOR + CellVisibility.quote(label)); @@ -520,7 +521,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer + VisibilityConstants.NOT_OPERATOR + CellVisibility.quote(label)); } } else { - String label = Bytes.toString(tag.getBuffer(), offset, len); + String label = ByteBufferUtils.toString(tag.getBuffer(), offset, len); if (expressionStart) { visibilityString.append(VisibilityConstants.OPEN_PARAN + CellVisibility.quote(label)); } else { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java index 01e22da..dd45db9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java @@ -25,8 +25,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.security.PrivilegedExceptionAction; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; @@ -36,13 +34,12 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; @@ -60,8 +57,6 @@ import org.junit.experimental.categories.Category; @Category({ SecurityTests.class, MediumTests.class }) public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilityLabelsReplication { - private static final Log LOG = LogFactory - .getLog(TestVisibilityLabelReplicationWithExpAsString.class); @Override @Before @@ -198,7 +193,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit boolean foundNonVisTag = false; for(Tag t : TestCoprocessorForTagsAtSink.tags) { if(t.getType() == NON_VIS_TAG_TYPE) { - assertEquals(TEMP, Bytes.toString(t.getValue())); + assertEquals(TEMP, Bytes.toString(TagUtil.getTagValue(t))); foundNonVisTag = true; break; } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index c087f4e..d73617c 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -45,12 +46,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; -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; @@ -296,11 +297,10 @@ public class TestVisibilityLabelsReplication { for (Cell cell : cells) { if ((Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), row, 0, row.length))) { - List tags = Tag - .asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()); + List tags = TagUtil.asList(cell); for (Tag tag : tags) { if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) { - assertEquals(visTag, Bytes.toString(tag.getValue())); + assertEquals(visTag, Bytes.toString(TagUtil.getTagValue(tag))); tagFound = true; break; } @@ -346,7 +346,7 @@ public class TestVisibilityLabelsReplication { boolean foundNonVisTag = false; for (Tag t : TestCoprocessorForTagsAtSink.tags) { if (t.getType() == NON_VIS_TAG_TYPE) { - assertEquals(TEMP, Bytes.toString(t.getValue())); + assertEquals(TEMP, Bytes.toString(TagUtil.getTagValue(t))); foundNonVisTag = true; break; } @@ -430,11 +430,11 @@ public class TestVisibilityLabelsReplication { if (cf == null) { cf = kv.getFamily(); } - Tag tag = new Tag((byte) NON_VIS_TAG_TYPE, attribute); + Tag tag = new ArrayBackedTag((byte) NON_VIS_TAG_TYPE, attribute); List tagList = new ArrayList(); tagList.add(tag); tagList.addAll(kv.getTags()); - byte[] fromList = Tag.fromList(tagList); + byte[] fromList = TagUtil.fromList(tagList); TagRewriteCell newcell = new TagRewriteCell(kv, fromList); KeyValue newKV = new KeyValue(kv.getRow(), 0, kv.getRowLength(), kv.getFamily(), 0, kv.getFamilyLength(), kv.getQualifier(), 0, kv.getQualifierLength(), @@ -460,7 +460,7 @@ public class TestVisibilityLabelsReplication { // Check tag presence in the 1st cell in 1st Result if (!results.isEmpty()) { Cell cell = results.get(0); - tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength()); + tags = TagUtil.asList(cell); } } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java index 52f86c3..68ff8c4 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java @@ -65,6 +65,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili conf = TEST_UTIL.getConfiguration(); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); conf.setBoolean("hbase.online.schema.update.enable", true); + //conf.set("hbase.client.rpc.compressor", "org.apache.hadoop.io.compress.GzipCodec"); VisibilityTestUtil.enableVisiblityLabels(conf); conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class, ScanLabelGenerator.class); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java index b2d0ae5..134e14f 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java @@ -330,7 +330,7 @@ public class TestVisibilityLabelsWithDeletes { } @Test - public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersionsNoTimestamp() + public void testVisibilityLabelsWithDeleteColumnsWithMVNTS() throws Exception { setAuths(); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); @@ -379,7 +379,7 @@ public class TestVisibilityLabelsWithDeletes { @Test public void - testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMultipleVersionsNoTimestamp() + testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMVNoTS() throws Exception { setAuths(); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); @@ -496,6 +496,7 @@ public class TestVisibilityLabelsWithDeletes { put.setCellVisibility(new CellVisibility(SECRET)); table.put(put); TEST_UTIL.getHBaseAdmin().flush(tableName); + TEST_UTIL.getHBaseAdmin().flush(LABELS_TABLE_NAME); PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { @@ -536,6 +537,7 @@ public class TestVisibilityLabelsWithDeletes { } }; SUPERUSER.runAs(actiona); + TEST_UTIL.getHBaseAdmin().flush(tableName); s = new Scan(); s.setMaxVersions(5); s.setAuthorizations(new Authorizations(CONFIDENTIAL)); @@ -983,7 +985,7 @@ public class TestVisibilityLabelsWithDeletes { @Test - public void testDeleteColumnWithSpecificTimeStampUsingMultipleVersionsUnMatchingVisExpression() + public void testDeleteColumnWithSpecificTimeStampUsingMVs() throws Exception { setAuths(); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); @@ -1523,7 +1525,7 @@ public class TestVisibilityLabelsWithDeletes { } @Test - public void testDeleteFamilyLatestTimeStampWithMulipleVersionsWithoutCellVisibilityInPuts() + public void testDeleteFamilyLatestTimeStampWithMVNoCV() throws Exception { setAuths(); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); @@ -1977,7 +1979,8 @@ public class TestVisibilityLabelsWithDeletes { }; VisibilityLabelsResponse response = SUPERUSER.runAs(action); final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName()); - try (Table table = doPuts(tableName)){ + Table table = doPuts(tableName); + try { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { @@ -2056,6 +2059,11 @@ public class TestVisibilityLabelsWithDeletes { assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(), row1, 0, row1.length)); assertEquals(current.getTimestamp(), 124l); + TEST_UTIL.getHBaseAdmin().flush(tableName); + } finally { + if (table != null) { + table.close(); + } } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java index 70d6d9d..54b522f 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; @@ -77,7 +78,7 @@ public class LoadTestDataGeneratorWithTags extends DefaultDataGenerator { minTagLength + random.nextInt(maxTagLength - minTagLength)); tags = new ArrayList(); for (int n = 0; n < numTags; n++) { - tags.add(new Tag((byte) 127, tag)); + tags.add(new ArrayBackedTag((byte) 127, tag)); } Cell updatedCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java index 21d7490..28b5bfa 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java @@ -56,7 +56,7 @@ public class TestByteBloomFilter extends TestCase { bf1.add(bval, 1, bval.length-1); assertTrue( bf1.contains(bkey) ); - assertTrue( bf1.contains(bval, 1, bval.length-1) ); + assertTrue( bf1.contains(bval, 1, bval.length-1)); assertFalse( bf1.contains(bval) ); assertFalse( bf1.contains(bval) ); @@ -70,13 +70,63 @@ public class TestByteBloomFilter extends TestCase { assertTrue(newBf1.contains(key1, bb)); assertFalse(newBf1.contains(key2, bb)); assertTrue( newBf1.contains(bkey, bb) ); - assertTrue( newBf1.contains(bval, 1, bval.length-1, bb) ); + assertTrue( newBf1.contains(bval, 1, bval.length-1, new MultiByteBuffer(bb)) ); assertFalse( newBf1.contains(bval, bb) ); assertFalse( newBf1.contains(bval, bb) ); System.out.println("Serialized as " + bOut.size() + " bytes"); assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding } + + public void testBasicBloomWithBB() throws Exception { + ByteBloomFilter bf1 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); + ByteBloomFilter bf2 = new ByteBloomFilter(1000, (float)0.01, Hash.MURMUR_HASH, 0); + bf1.allocBloom(); + bf2.allocBloom(); + + // test 1: verify no fundamental false negatives or positives + byte[] b1 = {1,2,3,4,5,6,7,8,9}; + byte[] b2 = {1,2,3,4,5,6,7,8,7}; + ByteBuffer key1 = ByteBuffer.wrap(b1); + ByteBuffer key2 = ByteBuffer.wrap(b2); + + bf1.add(key1, 0, key1.limit()); + bf2.add(key2, 0 , key1.limit()); + + assertTrue(bf1.contains(key1, 0, key1.limit(), null)); + assertFalse(bf1.contains(key2, 0, key2.limit(), null)); + assertFalse(bf2.contains(key1, 0 , key1.limit(), null)); + assertTrue(bf2.contains(key2, 0 , key2.limit(), null)); + + ByteBuffer bkey = ByteBuffer.wrap(new byte[]{1,2,3,4}); + byte [] bvalarray = "this is a much larger byte array".getBytes(); + ByteBuffer bval = ByteBuffer.wrap(bvalarray); + + bf1.add(bkey, 0, bkey.limit()); + bf1.add(bval, 1, bval.limit()-1); + + assertTrue( bf1.contains(bkey, 0, bkey.limit(), null) ); + assertTrue( bf1.contains(bval, 1, bval.limit()-1, null) ); + assertFalse( bf1.contains(bval, 0, bval.limit(), null) ); + assertFalse( bf1.contains(bval, 0, bval.limit(), null) ); + + // test 2: serialization & deserialization. + // (convert bloom to byte array & read byte array back in as input) + ByteArrayOutputStream bOut = new ByteArrayOutputStream(); + bf1.writeBloom(new DataOutputStream(bOut)); + ByteBuffer bb = ByteBuffer.wrap(bOut.toByteArray()); + ByteBloomFilter newBf1 = new ByteBloomFilter(1000, (float)0.01, + Hash.MURMUR_HASH, 0); + assertTrue(newBf1.contains(key1, 0, key1.limit(), new MultiByteBuffer(bb))); + assertFalse(newBf1.contains(key2, 0, key2.limit(), new MultiByteBuffer(bb))); + assertTrue( newBf1.contains(bkey, 0, bkey.limit(), new MultiByteBuffer(bb)) ); + assertTrue( newBf1.contains(bval, 1, bval.limit()-1, new MultiByteBuffer(bb)) ); + assertFalse( newBf1.contains(bval, 0, bval.limit(), new MultiByteBuffer(bb)) ); + assertFalse( newBf1.contains(bval, 0, bval.limit(), new MultiByteBuffer(bb)) ); + + System.out.println("Serialized as " + bOut.size() + " bytes"); + assertTrue(bOut.size() - bf1.byteSize < 10); //... allow small padding + } public void testBloomFold() throws Exception { // test: foldFactor < log(max/actual) diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java index 8a48d32..17a2537 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferUtils.java @@ -323,4 +323,13 @@ public class TestByteBufferUtils { assertEquals(5, buffer.position()); assertEquals(5, buffer.limit()); } + @Test + public void copyFromBufferToBytes() { + ByteBuffer buffer = ByteBuffer.allocateDirect(8); + buffer.putLong(10); + byte[] dest = new byte[9]; + ByteBufferUtils.copyFromBufferToByteArray(dest, buffer, 0, 1, 8); + long res = Bytes.toLong(dest, 1); + assertEquals(res, 10); + } }