diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index 3661c063e88c..89e91ca80361 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -33,7 +33,6 @@
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.RawComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -87,22 +86,6 @@ public class KeyValue implements ExtendedCell, Cloneable {
public static final byte[] COLUMN_FAMILY_DELIM_ARRAY = new byte[] { COLUMN_FAMILY_DELIMITER };
- /**
- * Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion of
- * KeyValue only.
- * @deprecated Use {@link CellComparator#getInstance()} instead. Deprecated for hbase 2.0, remove
- * for hbase 3.0.
- */
- @Deprecated
- public static final KVComparator COMPARATOR = new KVComparator();
- /**
- * A {@link KVComparator} for hbase:meta catalog table {@link KeyValue}s.
- * @deprecated Use {@link MetaCellComparator#META_COMPARATOR} instead. Deprecated for hbase 2.0,
- * remove for hbase 3.0.
- */
- @Deprecated
- public static final KVComparator META_COMPARATOR = new MetaComparator();
-
/** Size of the key length field in bytes */
public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
@@ -1497,623 +1480,6 @@ public static int getDelimiterInReverse(final byte[] b, final int offset, final
return result;
}
- /**
- * A {@link KVComparator} for hbase:meta catalog table {@link KeyValue}s.
- * @deprecated : {@link MetaCellComparator#META_COMPARATOR} to be used. Deprecated for hbase 2.0,
- * remove for hbase 3.0.
- */
- @Deprecated
- public static class MetaComparator extends KVComparator {
- /**
- * Compare key portion of a {@link KeyValue} for keys in hbase:meta table.
- */
- @Override
- public int compare(final Cell left, final Cell right) {
- return PrivateCellUtil.compareKeyIgnoresMvcc(MetaCellComparator.META_COMPARATOR, left, right);
- }
-
- @Override
- public int compareOnlyKeyPortion(Cell left, Cell right) {
- return compare(left, right);
- }
-
- @Override
- public int compareRows(byte[] 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 = Bytes.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 = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
- right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
- return result;
- }
-
- /**
- * Don't do any fancy Block Index splitting tricks.
- */
- @Override
- public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
- return Arrays.copyOf(rightKey, rightKey.length);
- }
-
- /**
- * The HFileV2 file format's trailer contains this class name. We reinterpret this and
- * instantiate the appropriate comparator. TODO: With V3 consider removing this.
- * @return legacy class name for FileFileTrailer#comparatorClassName
- */
- @Override
- public String getLegacyKeyComparatorName() {
- return "org.apache.hadoop.hbase.KeyValue$MetaKeyComparator";
- }
-
- @Override
- protected MetaComparator clone() throws CloneNotSupportedException {
- return (MetaComparator) super.clone();
- }
-
- /**
- * Override the row key comparison to parse and compare the meta row key parts.
- */
- @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);
- }
- }
-
- /**
- * Compare KeyValues. When we compare KeyValues, we only compare the Key portion. This means two
- * KeyValues with same Key but different Values are considered the same as far as this Comparator
- * is concerned.
- * @deprecated : Use {@link CellComparatorImpl}. Deprecated for hbase 2.0, remove for hbase 3.0.
- */
- @Deprecated
- public static class KVComparator implements RawComparator, SamePrefixComparator {
-
- /**
- * The HFileV2 file format's trailer contains this class name. We reinterpret this and
- * instantiate the appropriate comparator. TODO: With V3 consider removing this.
- * @return legacy class name for FileFileTrailer#comparatorClassName
- */
- public String getLegacyKeyComparatorName() {
- return "org.apache.hadoop.hbase.KeyValue$KeyComparator";
- }
-
- @Override // RawComparator
- public int compare(byte[] l, int loff, int llen, byte[] r, int roff, int rlen) {
- return compareFlatKey(l, loff, llen, r, roff, rlen);
- }
-
- /**
- * Compares the only the user specified portion of a Key. This is overridden by MetaComparator.
- * @param left left cell to compare row key
- * @param right right cell to compare row key
- * @return 0 if equal, <0 if left smaller, >0 if right smaller
- */
- protected int compareRowKey(final Cell left, final Cell right) {
- return CellComparatorImpl.COMPARATOR.compareRows(left, right);
- }
-
- /**
- * 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 the left kv serialized byte[] to be compared with
- * @param loffset the offset in the left byte[]
- * @param llength the length in the left byte[]
- * @param right the right kv serialized byte[] to be compared with
- * @param roffset the offset in the right byte[]
- * @param rlength the length in the right byte[]
- * @return 0 if equal, <0 if left smaller, >0 if right smaller
- */
- public int compareFlatKey(byte[] left, int loffset, int llength, byte[] right, int roffset,
- int rlength) {
- // Compare row
- short lrowlength = Bytes.toShort(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(byte[] left, byte[] right) {
- return compareFlatKey(left, 0, left.length, right, 0, right.length);
- }
-
- // compare a key against row/fam/qual/ts/type
- public int compareKey(Cell cell, byte[] row, int roff, int rlen, byte[] fam, int foff, int flen,
- byte[] col, int coff, int clen, long ts, byte type) {
-
- int compare =
- compareRows(cell.getRowArray(), 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;
- }
-
- compare = compareFamilies(cell.getFamilyArray(), cell.getFamilyOffset(),
- cell.getFamilyLength(), fam, foff, flen);
- if (compare != 0) {
- return compare;
- }
- compare = compareColumns(cell.getQualifierArray(), 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 PrivateCellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, right);
- }
-
- /**
- * 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 = CellComparatorImpl.COMPARATOR.compare(left, right);
- return compare;
- }
-
- public int compareTimestamps(final Cell left, final Cell right) {
- return CellComparatorImpl.COMPARATOR.compareTimestamps(left, right);
- }
-
- /**
- * Compares the rows of a cell
- * @param left left cell to compare rows for
- * @param right right cell to compare rows for
- * @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());
- }
-
- /**
- * Get the b[],o,l for left and right rowkey portions and compare.
- * @param left the left kv serialized byte[] to be compared with
- * @param loffset the offset in the left byte[]
- * @param llength the length in the left byte[]
- * @param right the right kv serialized byte[] to be compared with
- * @param roffset the offset in the right byte[]
- * @param rlength the length in the right byte[]
- * @return 0 if equal, <0 if left smaller, >0 if right smaller
- */
- public int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
- int rlength) {
- return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
- }
-
- int compareColumns(final Cell left, final short lrowlength, final Cell right,
- final short rrowlength) {
- return CellComparatorImpl.COMPARATOR.compareColumns(left, right);
- }
-
- protected int compareColumns(byte[] left, int loffset, int llength, final int lfamilylength,
- byte[] right, int roffset, int rlength, final int rfamilylength) {
- // Compare family portion first.
- int diff = Bytes.compareTo(left, loffset, lfamilylength, right, roffset, rfamilylength);
- if (diff != 0) {
- return diff;
- }
- // Compare qualifier portion
- return Bytes.compareTo(left, loffset + lfamilylength, llength - lfamilylength, right,
- roffset + rfamilylength, rlength - rfamilylength);
- }
-
- static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
- // The below older timestamps sorting ahead of newer timestamps looks
- // wrong but it is intentional. This way, newer timestamps are first
- // found when we iterate over a memstore and newer versions are the
- // first we trip over when reading from a store file.
- if (ltimestamp < rtimestamp) {
- return 1;
- } else if (ltimestamp > rtimestamp) {
- return -1;
- }
- return 0;
- }
-
- /**
- * Overridden
- * @param commonPrefix location of expected common prefix
- * @param left the left kv serialized byte[] to be compared with
- * @param loffset the offset in the left byte[]
- * @param llength the length in the left byte[]
- * @param right the right kv serialized byte[] to be compared with
- * @param roffset the offset in the byte[]
- * @param rlength the length in the right byte[]
- * @return 0 if equal, <0 if left smaller, >0 if right smaller
- */
- @Override // SamePrefixComparator
- public int compareIgnoringPrefix(int commonPrefix, byte[] left, int loffset, int llength,
- byte[] right, int roffset, int rlength) {
- // Compare row
- short lrowlength = Bytes.toShort(left, loffset);
- short rrowlength;
-
- int comparisonResult = 0;
- if (commonPrefix < ROW_LENGTH_SIZE) {
- // almost nothing in common
- rrowlength = Bytes.toShort(right, roffset);
- comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE, lrowlength, right,
- roffset + ROW_LENGTH_SIZE, rrowlength);
- } else { // the row length is the same
- rrowlength = lrowlength;
- if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
- // The rows are not the same. Exclude the common prefix and compare
- // the rest of the two rows.
- int common = commonPrefix - ROW_LENGTH_SIZE;
- comparisonResult = compareRows(left, loffset + common + ROW_LENGTH_SIZE,
- lrowlength - common, right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
- }
- }
- if (comparisonResult != 0) {
- return comparisonResult;
- }
-
- assert lrowlength == rrowlength;
- return compareWithoutRow(commonPrefix, left, loffset, llength, right, roffset, rlength,
- lrowlength);
- }
-
- /**
- * Compare columnFamily, qualifier, timestamp, and key type (everything except the row). This
- * method is used both in the normal comparator and the "same-prefix" comparator. Note that we
- * are assuming that row portions of both KVs have already been parsed and found identical, and
- * we don't validate that assumption here. the length of the common prefix of the two key-values
- * being compared, including row length and row
- */
- private int compareWithoutRow(int commonPrefix, byte[] 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[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[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 Bytes.compareTo(left, lfamilyoffset + common, lfamilylength - common, right,
- rfamilyoffset + common, rfamilylength - common);
- }
- // Compare family & qualifier together.
- final int comparison = Bytes.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 = 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);
- }
-
- 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 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;
- }
-
- /**
- * Compares the row and column of two keyvalues for equality
- * @param left left cell to compare row and column
- * @param right right cell to compare row and column
- * @return True if same row and column.
- */
- public boolean matchingRowColumn(final Cell left, final Cell right) {
- short lrowlength = left.getRowLength();
- short rrowlength = right.getRowLength();
-
- // 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())
- ) {
- return false;
- }
-
- if (!matchingRows(left, lrowlength, right, rrowlength)) {
- return false;
- }
-
- 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);
- if (diff != 0) {
- return false;
- } else {
- diff = compareColumns(left.getQualifierArray(), left.getQualifierOffset(), lclength,
- right.getQualifierArray(), right.getQualifierOffset(), rclength);
- return diff == 0;
- }
- }
-
- /**
- * Compares the row of two keyvalues for equality
- * @param left left cell to compare row
- * @param right right cell to compare row
- * @return True if rows match.
- */
- public boolean matchingRows(final Cell left, final Cell right) {
- short lrowlength = left.getRowLength();
- short rrowlength = right.getRowLength();
- return matchingRows(left, lrowlength, right, rrowlength);
- }
-
- /**
- * Compares the row of two keyvalues for equality
- * @param left left cell to compare row
- * @param lrowlength left row length
- * @param right right cell to compare row
- * @param rrowlength right row length
- * @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);
- }
-
- /**
- * Compare rows. Just calls Bytes.equals, but it's good to have this encapsulated.
- * @param left Left row array.
- * @param loffset Left row offset.
- * @param llength Left row length.
- * @param right Right row array.
- * @param roffset Right row offset.
- * @param rlength Right row length.
- * @return Whether rows are the same row.
- */
- public boolean matchingRows(final byte[] left, final int loffset, final int llength,
- final byte[] right, final int roffset, final int rlength) {
- return Bytes.equals(left, loffset, llength, right, roffset, rlength);
- }
-
- public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
- byte[] fakeKey = getShortMidpointKey(lastKeyOfPreviousBlock, firstKeyInBlock);
- if (compareFlatKey(fakeKey, firstKeyInBlock) > 0) {
- LOG.error("Unexpected getShortMidpointKey result, fakeKey:" + Bytes.toStringBinary(fakeKey)
- + ", firstKeyInBlock:" + Bytes.toStringBinary(firstKeyInBlock));
- return firstKeyInBlock;
- }
- if (lastKeyOfPreviousBlock != null && compareFlatKey(lastKeyOfPreviousBlock, fakeKey) >= 0) {
- LOG.error("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:"
- + Bytes.toStringBinary(lastKeyOfPreviousBlock) + ", fakeKey:"
- + Bytes.toStringBinary(fakeKey));
- return firstKeyInBlock;
- }
- return fakeKey;
- }
-
- /**
- * This is a HFile block index key optimization.
- * @param leftKey byte array for left Key
- * @param rightKey byte array for right Key
- * @return 0 if equal, <0 if left smaller, >0 if right smaller
- * @deprecated Since 0.99.2;
- */
- @Deprecated
- public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
- if (rightKey == null) {
- throw new IllegalArgumentException("rightKey can not be null");
- }
- if (leftKey == null) {
- return Arrays.copyOf(rightKey, rightKey.length);
- }
- if (compareFlatKey(leftKey, rightKey) >= 0) {
- throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
- + ", rightKey:" + Bytes.toString(rightKey));
- }
-
- short leftRowLength = Bytes.toShort(leftKey, 0);
- short rightRowLength = Bytes.toShort(rightKey, 0);
- int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
- int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
- int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
- int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
- int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
- int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
- // rows are equal
- if (
- leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
- rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0
- ) {
- // Compare family & qualifier together.
- int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
- rightCommonLength, rightColumnLength);
- // same with "row + family + qualifier", return rightKey directly
- if (comparison == 0) {
- return Arrays.copyOf(rightKey, rightKey.length);
- }
- // "family + qualifier" are different, generate a faked key per rightKey
- byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
- Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
- Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
- return newKey;
- }
- // rows are different
- short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
- short diffIdx = 0;
- while (
- diffIdx < minLength
- && leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]
- ) {
- diffIdx++;
- }
- byte[] newRowKey = null;
- if (diffIdx >= minLength) {
- // leftKey's row is prefix of rightKey's.
- newRowKey = new byte[diffIdx + 1];
- System.arraycopy(rightKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx + 1);
- } else {
- int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
- if (
- (0xff & diffByte) < 0xff && (diffByte + 1) < (rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)
- ) {
- newRowKey = new byte[diffIdx + 1];
- System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
- newRowKey[diffIdx] = (byte) (diffByte + 1);
- } else {
- newRowKey = new byte[diffIdx + 1];
- System.arraycopy(rightKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx + 1);
- }
- }
- return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP, Type.Maximum)
- .getKey();
- }
-
- @Override
- protected KVComparator clone() throws CloneNotSupportedException {
- return (KVComparator) super.clone();
- }
-
- }
-
/**
* Create a KeyValue reading from in
* @param in Where to read bytes from. Creates a byte array to hold the KeyValue backing bytes
@@ -2166,28 +1532,6 @@ public static long write(final KeyValue kv, final DataOutput out) throws IOExcep
return (long) length + Bytes.SIZEOF_INT;
}
- /**
- * Write out a KeyValue in the manner in which we used to when KeyValue was a Writable but do not
- * require a {@link DataOutput}, just take plain {@link OutputStream} Named oswrite
- * so does not clash with {@link #write(KeyValue, DataOutput)}
- * @param kv the KeyValue on which write is being requested
- * @param out OutputStream to write keyValue to
- * @param withTags boolean value indicating write is with Tags or not
- * @return Length written on stream
- * @throws IOException if any IO error happen
- * @see #create(DataInput) for the inverse function
- * @see #write(KeyValue, DataOutput)
- * @see KeyValueUtil#oswrite(Cell, OutputStream, boolean)
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
- * {@link #write(OutputStream, boolean)}
- */
- @Deprecated
- public static long oswrite(final KeyValue kv, final OutputStream out, final boolean withTags)
- throws IOException {
- ByteBufferUtils.putInt(out, kv.getSerializedSize(withTags));
- return (long) kv.write(out, withTags) + Bytes.SIZEOF_INT;
- }
-
@Override
public int write(OutputStream out, boolean withTags) throws IOException {
int len = getSerializedSize(withTags);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index 861a68970577..72340eeead4f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -27,7 +27,6 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.InnerStoreCellComparator;
-import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MetaCellComparator;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.util.Bytes;
@@ -555,16 +554,10 @@ private static Class extends CellComparator> getComparatorClass(String compara
throws IOException {
Class extends CellComparator> comparatorKlass;
// for BC
- if (
- comparatorClassName.equals(KeyValue.COMPARATOR.getLegacyKeyComparatorName())
- || comparatorClassName.equals(KeyValue.COMPARATOR.getClass().getName())
- || (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator"))
- ) {
+ if (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator")) {
comparatorKlass = InnerStoreCellComparator.class;
} else if (
- comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())
- || comparatorClassName.equals(KeyValue.META_COMPARATOR.getClass().getName())
- || (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator$MetaCellComparator"))
+ comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator$MetaCellComparator")
|| (comparatorClassName
.equals("org.apache.hadoop.hbase.CellComparatorImpl$MetaCellComparator"))
|| (comparatorClassName.equals("org.apache.hadoop.hbase.MetaCellComparator"))
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
index 98a3c01a8e43..dc4ecd1ff8e6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
@@ -46,6 +46,7 @@
import org.apache.hadoop.hbase.io.TimeRange;
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.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.io.DataInputBuffer;
@@ -102,8 +103,10 @@ public void testCreateKeyValueInvalidNegativeLength() {
long l = 0;
try {
- l = KeyValue.oswrite(kv_0, dos, false);
- l += KeyValue.oswrite(kv_1, dos, false);
+ ByteBufferUtils.putInt(dos, kv_0.getSerializedSize(false));
+ l = (long) kv_0.write(dos, false) + Bytes.SIZEOF_INT;
+ ByteBufferUtils.putInt(dos, kv_1.getSerializedSize(false));
+ l += (long) kv_1.write(dos, false) + Bytes.SIZEOF_INT;
assertEquals(100L, l);
} catch (IOException e) {
fail("Unexpected IOException" + e.getMessage());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
index 3bad8d46a149..5c4767b2a677 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestFixedFileTrailer.java
@@ -41,7 +41,6 @@
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.InnerStoreCellComparator;
-import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MetaCellComparator;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -101,16 +100,8 @@ public void setUp() throws IOException {
@Test
public void testCreateComparator() throws IOException {
- assertEquals(InnerStoreCellComparator.class,
- createComparator(KeyValue.COMPARATOR.getLegacyKeyComparatorName()).getClass());
- assertEquals(InnerStoreCellComparator.class,
- createComparator(KeyValue.COMPARATOR.getClass().getName()).getClass());
assertEquals(InnerStoreCellComparator.class,
createComparator(CellComparator.class.getName()).getClass());
- assertEquals(MetaCellComparator.class,
- createComparator(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName()).getClass());
- assertEquals(MetaCellComparator.class,
- createComparator(KeyValue.META_COMPARATOR.getClass().getName()).getClass());
assertEquals(MetaCellComparator.class,
createComparator("org.apache.hadoop.hbase.CellComparator$MetaCellComparator").getClass());
assertEquals(MetaCellComparator.class,
|