HBASE-10800 - Use CellComparator instead of KVComparator (Ram)
This commit is contained in:
parent
25b5098da7
commit
977f867439
|
@ -27,10 +27,10 @@ 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.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
|
@ -842,7 +842,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
@Deprecated
|
||||
public KVComparator getComparator() {
|
||||
return isMetaRegion()?
|
||||
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
|
||||
KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,13 +31,12 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
|
||||
/**
|
||||
* A cache implementation for region locations from meta.
|
||||
|
@ -96,9 +95,9 @@ public class MetaCache {
|
|||
return null;
|
||||
}
|
||||
|
||||
private KVComparator getRowComparator(TableName tableName) {
|
||||
return TableName.META_TABLE_NAME.equals(tableName) ? KeyValue.META_COMPARATOR
|
||||
: KeyValue.COMPARATOR;
|
||||
private CellComparator getRowComparator(TableName tableName) {
|
||||
return TableName.META_TABLE_NAME.equals(tableName) ? CellComparator.META_COMPARATOR
|
||||
: CellComparator.COMPARATOR;
|
||||
}
|
||||
/**
|
||||
* Put a newly discovered HRegionLocation into the cache.
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.NavigableMap;
|
|||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -204,14 +205,14 @@ public class Result implements CellScannable, CellScanner {
|
|||
* Return the array of Cells backing this Result instance.
|
||||
*
|
||||
* The array is sorted from smallest -> largest using the
|
||||
* {@link KeyValue#COMPARATOR}.
|
||||
* {@link CellComparator#COMPARATOR}.
|
||||
*
|
||||
* The array only contains what your Get or Scan specifies and no more.
|
||||
* For example if you request column "A" 1 version you will have at most 1
|
||||
* Cell in the array. If you request column "A" with 2 version you will
|
||||
* have at most 2 Cells, with the first one being the newer timestamp and
|
||||
* the second being the older timestamp (this is the sort order defined by
|
||||
* {@link KeyValue#COMPARATOR}). If columns don't exist, they won't be
|
||||
* {@link CellComparator#COMPARATOR}). If columns don't exist, they won't be
|
||||
* present in the result. Therefore if you ask for 1 version all columns,
|
||||
* it is safe to iterate over this array and expect to see 1 Cell for
|
||||
* each column and no more.
|
||||
|
@ -237,7 +238,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
|
||||
/**
|
||||
* Return the Cells for the specific column. The Cells are sorted in
|
||||
* the {@link KeyValue#COMPARATOR} order. That implies the first entry in
|
||||
* the {@link CellComparator#COMPARATOR} order. That implies the first entry in
|
||||
* the list is the most recent column. If the query (Scan or Get) only
|
||||
* requested 1 version the list will contain at most 1 entry. If the column
|
||||
* did not exist in the result set (either the column does not exist
|
||||
|
@ -282,7 +283,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
family, qualifier);
|
||||
|
||||
// pos === ( -(insertion point) - 1)
|
||||
int pos = Arrays.binarySearch(kvs, searchTerm, KeyValue.COMPARATOR);
|
||||
int pos = Arrays.binarySearch(kvs, searchTerm, CellComparator.COMPARATOR);
|
||||
// never will exact match
|
||||
if (pos < 0) {
|
||||
pos = (pos+1) * -1;
|
||||
|
@ -327,7 +328,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
qualifier, qoffset, qlength);
|
||||
|
||||
// pos === ( -(insertion point) - 1)
|
||||
int pos = Arrays.binarySearch(kvs, searchTerm, KeyValue.COMPARATOR);
|
||||
int pos = Arrays.binarySearch(kvs, searchTerm, CellComparator.COMPARATOR);
|
||||
// never will exact match
|
||||
if (pos < 0) {
|
||||
pos = (pos+1) * -1;
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -216,7 +216,7 @@ final public class FilterList extends Filter {
|
|||
|
||||
@Override
|
||||
public Cell transformCell(Cell c) throws IOException {
|
||||
if (!CellComparator.equals(c, referenceCell)) {
|
||||
if (!CellUtil.equals(c, referenceCell)) {
|
||||
throw new IllegalStateException("Reference Cell: " + this.referenceCell + " does not match: "
|
||||
+ c);
|
||||
}
|
||||
|
@ -415,7 +415,7 @@ final public class FilterList extends Filter {
|
|||
keyHint = curKeyHint;
|
||||
continue;
|
||||
}
|
||||
if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
|
||||
if (CellComparator.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
|
||||
keyHint = curKeyHint;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import java.util.Stack;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
|
@ -294,7 +293,7 @@ public class ParseFilter {
|
|||
* @return an ArrayList containing the arguments of the filter in the filter string
|
||||
*/
|
||||
public static ArrayList<byte []> getFilterArguments (byte [] filterStringAsByteArray) {
|
||||
int argumentListStartIndex = KeyValue.getDelimiter(filterStringAsByteArray, 0,
|
||||
int argumentListStartIndex = Bytes.searchDelimiterIndex(filterStringAsByteArray, 0,
|
||||
filterStringAsByteArray.length,
|
||||
ParseConstants.LPAREN);
|
||||
if (argumentListStartIndex == -1) {
|
||||
|
@ -818,7 +817,8 @@ public class ParseFilter {
|
|||
* @return the parsed arguments of the comparator as a 2D byte array
|
||||
*/
|
||||
public static byte [][] parseComparator (byte [] comparator) {
|
||||
final int index = KeyValue.getDelimiter(comparator, 0, comparator.length, ParseConstants.COLON);
|
||||
final int index = Bytes.searchDelimiterIndex(comparator, 0, comparator.length,
|
||||
ParseConstants.COLON);
|
||||
if (index == -1) {
|
||||
throw new IllegalArgumentException("Incorrect comparator");
|
||||
}
|
||||
|
|
|
@ -32,12 +32,14 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
|
||||
import org.apache.commons.lang.NotImplementedException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -642,7 +644,7 @@ public class TestClientNoCluster extends Configured implements Tool {
|
|||
* Comparator for meta row keys.
|
||||
*/
|
||||
private static class MetaRowsComparator implements Comparator<byte []> {
|
||||
private final KeyValue.KVComparator delegate = new KeyValue.MetaComparator();
|
||||
private final CellComparator delegate = CellComparator.META_COMPARATOR;
|
||||
@Override
|
||||
public int compare(byte[] left, byte[] right) {
|
||||
return delegate.compareRows(left, 0, left.length, right, 0, right.length);
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -387,7 +388,7 @@ public class TestOperation {
|
|||
Assert.assertEquals(1984L, c.get(0).getTimestamp());
|
||||
Assert.assertArrayEquals(VALUE, CellUtil.cloneValue(c.get(0)));
|
||||
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
|
||||
Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
|
||||
p = new Put(ROW);
|
||||
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2013L, null);
|
||||
|
@ -396,7 +397,7 @@ public class TestOperation {
|
|||
Assert.assertEquals(2013L, c.get(0).getTimestamp());
|
||||
Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
|
||||
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
|
||||
Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
|
||||
p = new Put(ByteBuffer.wrap(ROW));
|
||||
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
|
||||
|
@ -406,7 +407,7 @@ public class TestOperation {
|
|||
Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
|
||||
Assert.assertArrayEquals(ROW, CellUtil.cloneRow(c.get(0)));
|
||||
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
|
||||
Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
|
||||
p = new Put(ByteBuffer.wrap(ROW), 1970L);
|
||||
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
|
||||
|
@ -416,7 +417,7 @@ public class TestOperation {
|
|||
Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
|
||||
Assert.assertArrayEquals(ROW, CellUtil.cloneRow(c.get(0)));
|
||||
Assert.assertEquals(1970L, p.getTimeStamp());
|
||||
Assert.assertEquals(0, KeyValue.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.hbase;
|
|||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -34,6 +36,9 @@ import com.google.common.primitives.Longs;
|
|||
* takes account of the special formatting of the row where we have commas to delimit table from
|
||||
* regionname, from row. See KeyValue for how it has a special comparator to do hbase:meta cells
|
||||
* and yet another for -ROOT-.
|
||||
* While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells format
|
||||
* should be taken into consideration, for which the instance of this comparator
|
||||
* should be used. In all other cases the static APIs in this comparator would be enough
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
|
||||
value="UNKNOWN",
|
||||
|
@ -41,24 +46,74 @@ import com.google.common.primitives.Longs;
|
|||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class CellComparator implements Comparator<Cell>, Serializable {
|
||||
static final Log LOG = LogFactory.getLog(CellComparator.class);
|
||||
private static final long serialVersionUID = -8760041766259623329L;
|
||||
|
||||
/**
|
||||
* Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion
|
||||
* of KeyValue only.
|
||||
*/
|
||||
public static final CellComparator COMPARATOR = new CellComparator();
|
||||
/**
|
||||
* A {@link CellComparator} for <code>hbase:meta</code> catalog table
|
||||
* {@link KeyValue}s.
|
||||
*/
|
||||
public static final CellComparator META_COMPARATOR = new MetaCellComparator();
|
||||
|
||||
@Override
|
||||
public int compare(Cell a, Cell b) {
|
||||
return compare(a, b, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares only the key portion of a cell. It does not include the sequence id/mvcc of the
|
||||
* cell
|
||||
* @param left
|
||||
* @param right
|
||||
* @return an int greater than 0 if left > than right
|
||||
* lesser than 0 if left < than right
|
||||
* equal to 0 if left is equal to right
|
||||
*/
|
||||
public final int compareKeyIgnoresMvcc(Cell left, Cell right) {
|
||||
return compare(left, right, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used when a cell needs to be compared with a key byte[] such as cases of
|
||||
* finding the index from the index block, bloom keys from the bloom blocks
|
||||
* This byte[] is expected to be serialized in the KeyValue serialization format
|
||||
* If the KeyValue (Cell's) serialization format changes this method cannot be used.
|
||||
* @param left the cell to be compared
|
||||
* @param key the serialized key part of a KeyValue
|
||||
* @param offset the offset in the key byte[]
|
||||
* @param length the length of the key byte[]
|
||||
* @return an int greater than 0 if left is greater than right
|
||||
* lesser than 0 if left is lesser than right
|
||||
* equal to 0 if left is equal to right
|
||||
* TODO : We will be moving over to
|
||||
* compare(Cell, Cell) so that the key is also converted to a cell
|
||||
*/
|
||||
public final int compare(Cell left, byte[] key, int offset, int length) {
|
||||
// row
|
||||
short rrowlength = Bytes.toShort(key, offset);
|
||||
int c = compareRows(left, key, offset + Bytes.SIZEOF_SHORT, rrowlength);
|
||||
if (c != 0) return c;
|
||||
|
||||
// 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(left, key, offset, length, rrowlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare cells.
|
||||
* TODO: Replace with dynamic rather than static comparator so can change comparator
|
||||
* implementation.
|
||||
* @param a
|
||||
* @param b
|
||||
* @param ignoreSequenceid True if we are to compare the key portion only and ignore
|
||||
* the sequenceid. Set to false to compare key and consider sequenceid.
|
||||
* @return 0 if equal, -1 if a < b, and +1 if a > b.
|
||||
*/
|
||||
public static int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
|
||||
private final int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
|
||||
// row
|
||||
int c = compareRows(a, b);
|
||||
if (c != 0) return c;
|
||||
|
@ -75,292 +130,481 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
}
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
private static int findCommonPrefix(byte[] left, byte[] right, int leftLength, int rightLength,
|
||||
int leftOffset, int rightOffset) {
|
||||
int length = Math.min(leftLength, rightLength);
|
||||
int result = 0;
|
||||
|
||||
while (result < length && left[leftOffset + result] == right[rightOffset + result]) {
|
||||
result++;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
/**************** equals ****************************/
|
||||
|
||||
public static boolean equals(Cell a, Cell b){
|
||||
return equalsRow(a, b)
|
||||
&& equalsFamily(a, b)
|
||||
&& equalsQualifier(a, b)
|
||||
&& equalsTimestamp(a, b)
|
||||
&& 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 equalsFamily(Cell a, Cell b){
|
||||
return Bytes.equals(
|
||||
a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(),
|
||||
b.getFamilyArray(), 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 equalsTimestamp(Cell a, Cell b){
|
||||
return a.getTimestamp() == b.getTimestamp();
|
||||
}
|
||||
|
||||
public static boolean equalsType(Cell a, Cell b){
|
||||
return a.getTypeByte() == b.getTypeByte();
|
||||
}
|
||||
|
||||
public static int compareColumns(final Cell left, final Cell right) {
|
||||
/**
|
||||
* Compares the family and qualifier part of the cell
|
||||
* TODO : Handle BB cases here
|
||||
* @param left the left cell
|
||||
* @param right the right cell
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final 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 = compareFamilies(left.getFamilyArray(), lfoffset, lfamilylength,
|
||||
right.getFamilyArray(), rfoffset, rfamilylength);
|
||||
if (diff != 0) {
|
||||
return diff;
|
||||
} else {
|
||||
return compare(left.getQualifierArray(), left.getQualifierOffset(), lclength,
|
||||
return compareQualifiers(left.getQualifierArray(), left.getQualifierOffset(), lclength,
|
||||
right.getQualifierArray(), right.getQualifierOffset(), rclength);
|
||||
}
|
||||
}
|
||||
|
||||
public static int compareFamilies(Cell left, Cell right) {
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
/**
|
||||
* Compares the family and qualifier part of the cell
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lfamilyOffset
|
||||
* @param lfamilylength
|
||||
* @param lqualOffset
|
||||
* @param lQualLength
|
||||
* @param right
|
||||
* @param rfamilyOffset
|
||||
* @param rfamilylength
|
||||
* @param rqualOffset
|
||||
* @param rqualLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareColumns(final Cell left, int lfamilyOffset, int lfamilylength,
|
||||
int lqualOffset, int lQualLength, final Cell right, final int rfamilyOffset,
|
||||
final int rfamilylength, final int rqualOffset, int rqualLength) {
|
||||
int diff = compareFamilies(left.getFamilyArray(), lfamilyOffset, lfamilylength,
|
||||
right.getFamilyArray(), rfamilyOffset, rfamilylength);
|
||||
if (diff != 0) {
|
||||
return diff;
|
||||
} else {
|
||||
return compareQualifiers(left.getQualifierArray(), lqualOffset, lQualLength,
|
||||
right.getQualifierArray(), rqualOffset, rqualLength);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the family and qualifier part of a cell with a serialized Key value byte[]
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left the cell to be compared
|
||||
* @param lfamilyOffset
|
||||
* @param lfamilylength
|
||||
* @param lqualOffset
|
||||
* @param lQualLength
|
||||
* @param right the serialized key value byte array to be compared
|
||||
* @param rfamilyOffset
|
||||
* @param rfamilylength
|
||||
* @param rqualOffset
|
||||
* @param rqualLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareColumns(final Cell left, final int lfamilyOffset,
|
||||
final int lfamilylength, final int lqualOffset, final int lQualLength, final byte[] right,
|
||||
final int rfamilyOffset, final int rfamilylength, final int rqualOffset,
|
||||
final int rqualLength) {
|
||||
int diff = compareFamilies(left.getFamilyArray(), lfamilyOffset, lfamilylength, right,
|
||||
rfamilyOffset, rfamilylength);
|
||||
if (diff != 0) {
|
||||
return diff;
|
||||
} else {
|
||||
return compareQualifiers(left.getQualifierArray(), lqualOffset, lQualLength, right,
|
||||
rqualOffset, rqualLength);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare the families of left and right cell
|
||||
* TODO : Handle BB cases here
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareFamilies(Cell left, Cell right) {
|
||||
return compareFamilies(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
|
||||
public static int compareQualifiers(Cell left, Cell right) {
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
/**
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lOffset
|
||||
* @param lLength
|
||||
* @param right
|
||||
* @param rOffset
|
||||
* @param rLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareFamilies(Cell left, int lOffset, int lLength, Cell right,
|
||||
int rOffset, int rLength) {
|
||||
return compareFamilies(left.getFamilyArray(), lOffset, lLength, right.getFamilyArray(),
|
||||
rOffset, rLength);
|
||||
}
|
||||
|
||||
private final static int compareFamilies(Cell left, int lOffset, int lLength, byte[] right,
|
||||
int rOffset, int rLength) {
|
||||
return compareFamilies(left.getFamilyArray(), lOffset, lLength, right, rOffset, rLength);
|
||||
}
|
||||
|
||||
private final static int compareFamilies(byte[] leftFamily, int lFamOffset, int lFamLength,
|
||||
byte[] rightFamily, int rFamOffset, int rFamLen) {
|
||||
return Bytes.compareTo(leftFamily, lFamOffset, lFamLength, rightFamily, rFamOffset, rFamLen);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare the qualifiers part of the left and right cells.
|
||||
* TODO : Handle BB cases here
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareQualifiers(Cell left, Cell right) {
|
||||
return compareQualifiers(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
|
||||
right.getQualifierLength());
|
||||
}
|
||||
|
||||
public int compareFlatKey(Cell left, Cell right) {
|
||||
int compare = compareRows(left, right);
|
||||
if (compare != 0) {
|
||||
return compare;
|
||||
}
|
||||
return compareWithoutRow(left, right);
|
||||
/**
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lOffset
|
||||
* @param lLength
|
||||
* @param right
|
||||
* @param rOffset
|
||||
* @param rLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareQualifiers(Cell left, int lOffset, int lLength, Cell right,
|
||||
int rOffset, int rLength) {
|
||||
return compareQualifiers(left.getQualifierArray(), lOffset,
|
||||
lLength, right.getQualifierArray(), rOffset,
|
||||
rLength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Do not use comparing rows from hbase:meta. Meta table Cells have schema (table,startrow,hash)
|
||||
* so can't be treated as plain byte arrays as this method does.
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lOffset
|
||||
* @param lLength
|
||||
* @param right
|
||||
* @param rOffset
|
||||
* @param rLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public static int compareRows(final Cell left, final Cell right) {
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
private final static int compareQualifiers(Cell left, int lOffset, int lLength, byte[] right,
|
||||
int rOffset, int rLength) {
|
||||
return compareQualifiers(left.getQualifierArray(), lOffset,
|
||||
lLength, right, rOffset,
|
||||
rLength);
|
||||
}
|
||||
|
||||
private static int compareQualifiers(byte[] leftCol, int lColOffset, int lColLength,
|
||||
byte[] rightCol, int rColOffset, int rColLength) {
|
||||
return Bytes.compareTo(leftCol, lColOffset, lColLength, rightCol, rColOffset, rColLength);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
* TODO : we will have to handle BB cases here
|
||||
* @param commonPrefix
|
||||
* the length of the common prefix of the two key-values being
|
||||
* compared, including row length and row
|
||||
*/
|
||||
private final int compareWithoutRow(Cell left,
|
||||
byte[] right, int roffset, int rlength, short rowlength) {
|
||||
/***
|
||||
* KeyValue Format and commonLength:
|
||||
* |_keyLen_|_valLen_|_rowLen_|_rowKey_|_famiLen_|_fami_|_Quali_|....
|
||||
* ------------------|-------commonLength--------|--------------
|
||||
*/
|
||||
int commonLength = KeyValue.ROW_LENGTH_SIZE + KeyValue.FAMILY_LENGTH_SIZE + rowlength;
|
||||
|
||||
// commonLength + TIMESTAMP_TYPE_SIZE
|
||||
int commonLengthWithTSAndType = KeyValue.TIMESTAMP_TYPE_SIZE + commonLength;
|
||||
// ColumnFamily + Qualifier length.
|
||||
int lcolumnlength = left.getFamilyLength() + left.getQualifierLength();
|
||||
int rcolumnlength = rlength - commonLengthWithTSAndType;
|
||||
|
||||
byte ltype = left.getTypeByte();
|
||||
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 = left.getFamilyOffset();
|
||||
int rfamilyoffset = commonLength + roffset;
|
||||
|
||||
// Column family length.
|
||||
int lfamilylength = left.getFamilyLength();
|
||||
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);
|
||||
if (!sameFamilySize) {
|
||||
// comparing column family is enough.
|
||||
return compareFamilies(left, lfamilyoffset, lfamilylength, right,
|
||||
rfamilyoffset, rfamilylength);
|
||||
}
|
||||
// Compare family & qualifier together.
|
||||
// Families are same. Compare on qualifiers.
|
||||
int lQualOffset = left.getQualifierOffset();
|
||||
int lQualLength = left.getQualifierLength();
|
||||
int comparison = compareColumns(left, lfamilyoffset, lfamilylength, lQualOffset, lQualLength,
|
||||
right, rfamilyoffset, rfamilylength, rfamilyoffset + rfamilylength,
|
||||
(rcolumnlength - rfamilylength));
|
||||
if (comparison != 0) {
|
||||
return comparison;
|
||||
}
|
||||
|
||||
// //
|
||||
// Next compare timestamps.
|
||||
long rtimestamp = Bytes.toLong(right, roffset + (rlength - KeyValue.TIMESTAMP_TYPE_SIZE));
|
||||
int compare = compareTimestamps(left.getTimestamp(), 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the rows of the left and right cell
|
||||
* For the hbase:meta case the
|
||||
* ({@link #compareRows(byte[], int, int, byte[], int, int)} is overridden such
|
||||
* that it can handle the hbase:meta cells. The caller should ensure using the
|
||||
* appropriate comparator for hbase:meta
|
||||
* TODO : Handle BB cases here
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final int compareRows(final Cell left, final Cell right) {
|
||||
return compareRows(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the rows of two cells
|
||||
* We explicitly pass the offset and length details of the cell to avoid re-parsing
|
||||
* of the offset and length from the cell
|
||||
* @param left the cell to be compared
|
||||
* @param loffset the row offset of the left cell
|
||||
* @param llength the row length of the left cell
|
||||
* @param right the cell to be compared
|
||||
* @param roffset the row offset of the right cell
|
||||
* @param rlength the row length of the right cell
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final int compareRows(Cell left, int loffset, int llength, Cell right, int roffset,
|
||||
int rlength) {
|
||||
// TODO : for BB based cells all the hasArray based checks would happen
|
||||
// here. But we may have
|
||||
// to end up in multiple APIs accepting byte[] and BBs
|
||||
return compareRows(left.getRowArray(), loffset, llength, right.getRowArray(), roffset,
|
||||
rlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the row part of the cell with a simple plain byte[] like the
|
||||
* stopRow in Scan. This should be used with context where for hbase:meta
|
||||
* cells the {{@link #META_COMPARATOR} should be used
|
||||
*
|
||||
* @param left
|
||||
* the cell to be compared
|
||||
* @param right
|
||||
* the kv serialized byte[] to be compared with
|
||||
* @param roffset
|
||||
* the offset in the byte[]
|
||||
* @param rlength
|
||||
* the length in the byte[]
|
||||
* @return 0 if both cell and the byte[] are equal, 1 if the cell is bigger
|
||||
* than byte[], -1 otherwise
|
||||
*/
|
||||
public final int compareRows(Cell left, byte[] right, int roffset,
|
||||
int rlength) {
|
||||
// TODO : for BB based cells all the hasArray based checks would happen
|
||||
// here. But we may have
|
||||
// to end up in multiple APIs accepting byte[] and BBs
|
||||
return compareRows(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
/**
|
||||
* Do not use comparing rows from hbase:meta. Meta table Cells have schema (table,startrow,hash)
|
||||
* so can't be treated as plain byte arrays as this method does.
|
||||
*/
|
||||
public static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
|
||||
// TODO : CLEANUP : in order to do this we may have to modify some code
|
||||
// HRegion.next() and will involve a
|
||||
// Filter API change also. Better to do that later along with
|
||||
// HBASE-11425/HBASE-13387.
|
||||
public int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
|
||||
int rlength) {
|
||||
return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
|
||||
}
|
||||
|
||||
public static int compareWithoutRow(final Cell leftCell, final Cell rightCell) {
|
||||
private static int compareWithoutRow(final Cell left, final Cell right) {
|
||||
// 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.
|
||||
// Copied from KeyValue. This is bad in that we can't do memcmp w/ special rules like this.
|
||||
// TODO
|
||||
if (leftCell.getFamilyLength() + leftCell.getQualifierLength() == 0
|
||||
&& leftCell.getTypeByte() == Type.Minimum.getCode()) {
|
||||
int lFamLength = left.getFamilyLength();
|
||||
int rFamLength = right.getFamilyLength();
|
||||
int lQualLength = left.getQualifierLength();
|
||||
int rQualLength = right.getQualifierLength();
|
||||
if (lFamLength + lQualLength == 0
|
||||
&& left.getTypeByte() == Type.Minimum.getCode()) {
|
||||
// left is "bigger", i.e. it appears later in the sorted order
|
||||
return 1;
|
||||
}
|
||||
if (rightCell.getFamilyLength() + rightCell.getQualifierLength() == 0
|
||||
&& rightCell.getTypeByte() == Type.Minimum.getCode()) {
|
||||
if (rFamLength + rQualLength == 0
|
||||
&& right.getTypeByte() == Type.Minimum.getCode()) {
|
||||
return -1;
|
||||
}
|
||||
boolean sameFamilySize = (leftCell.getFamilyLength() == rightCell.getFamilyLength());
|
||||
boolean sameFamilySize = (lFamLength == rFamLength);
|
||||
int lFamOffset = left.getFamilyOffset();
|
||||
int rFamOffset = right.getFamilyOffset();
|
||||
if (!sameFamilySize) {
|
||||
// comparing column family is enough.
|
||||
|
||||
return Bytes.compareTo(leftCell.getFamilyArray(), leftCell.getFamilyOffset(),
|
||||
leftCell.getFamilyLength(), rightCell.getFamilyArray(), rightCell.getFamilyOffset(),
|
||||
rightCell.getFamilyLength());
|
||||
return compareFamilies(left, lFamOffset, lFamLength, right, rFamOffset, rFamLength);
|
||||
}
|
||||
int diff = compareColumns(leftCell, rightCell);
|
||||
// Families are same. Compare on qualifiers.
|
||||
int lQualOffset = left.getQualifierOffset();
|
||||
int rQualOffset = right.getQualifierOffset();
|
||||
int diff = compareColumns(left, lFamOffset, lFamLength, lQualOffset, lQualLength, right,
|
||||
rFamOffset, rFamLength, rQualOffset, rQualLength);
|
||||
if (diff != 0) return diff;
|
||||
|
||||
diff = compareTimestamps(leftCell, rightCell);
|
||||
diff = compareTimestamps(left, right);
|
||||
if (diff != 0) return diff;
|
||||
|
||||
// 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 & rightCell.getTypeByte()) - (0xff & leftCell.getTypeByte());
|
||||
return (0xff & right.getTypeByte()) - (0xff & left.getTypeByte());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare the timestamp of the left and right cell
|
||||
*
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if equal, -1 if left's ts is less than right's ts, 1 if left's ts
|
||||
* is greater than right's ts
|
||||
*/
|
||||
public static int compareTimestamps(final Cell left, final Cell right) {
|
||||
long ltimestamp = left.getTimestamp();
|
||||
long rtimestamp = right.getTimestamp();
|
||||
return compareTimestamps(ltimestamp, rtimestamp);
|
||||
return compareTimestamps(left.getTimestamp(), right.getTimestamp());
|
||||
}
|
||||
|
||||
/********************* hashCode ************************/
|
||||
|
||||
/**
|
||||
* Returns a hash code that is always the same for two Cells having a matching equals(..) result.
|
||||
* Used to compare two cells based on the column hint provided. This is specifically
|
||||
* used when we need to optimize the seeks based on the next indexed key. This is an
|
||||
* advance usage API specifically needed for some optimizations.
|
||||
* @param nextIndexedCell the next indexed cell
|
||||
* @param currentCell the cell to be compared
|
||||
* @param foff the family offset of the currentCell
|
||||
* @param flen the family length of the currentCell
|
||||
* @param colHint the column hint provided - could be null
|
||||
* @param coff the offset of the column hint if provided, if not offset of the currentCell's
|
||||
* qualifier
|
||||
* @param clen the length of the column hint if provided, if not length of the currentCell's
|
||||
* qualifier
|
||||
* @param ts the timestamp to be seeked
|
||||
* @param type the type to be seeked
|
||||
* @return an int based on the given column hint
|
||||
* TODO : To be moved out of here because this is a special API used in scan
|
||||
* optimization.
|
||||
*/
|
||||
public static int hashCode(Cell cell){
|
||||
if (cell == null) {// return 0 for empty Cell
|
||||
return 0;
|
||||
// compare a key against row/fam/qual/ts/type
|
||||
public final int compareKeyBasedOnColHint(Cell nextIndexedCell, Cell currentCell, int foff,
|
||||
int flen, byte[] colHint, int coff, int clen, long ts, byte type) {
|
||||
|
||||
int compare = 0;
|
||||
compare = compareRows(nextIndexedCell, nextIndexedCell.getRowOffset(),
|
||||
nextIndexedCell.getRowLength(), currentCell, currentCell.getRowOffset(),
|
||||
currentCell.getRowLength());
|
||||
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 (nextIndexedCell.getFamilyLength() + nextIndexedCell.getQualifierLength() == 0
|
||||
&& nextIndexedCell.getTypeByte() == Type.Minimum.getCode()) {
|
||||
// left is "bigger", i.e. it appears later in the sorted order
|
||||
return 1;
|
||||
}
|
||||
int qualLen = currentCell.getQualifierLength();
|
||||
if (flen + clen == 0 && type == Type.Minimum.getCode()) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
int hash = calculateHashForKeyValue(cell);
|
||||
hash = 31 * hash + (int)cell.getMvccVersion();
|
||||
return hash;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a hash code that is always the same for two Cells having a matching
|
||||
* equals(..) result. Note : Ignore mvcc while calculating the hashcode
|
||||
*
|
||||
* @param cell
|
||||
* @return hashCode
|
||||
*/
|
||||
public static int hashCodeIgnoreMvcc(Cell cell) {
|
||||
if (cell == null) {// return 0 for empty Cell
|
||||
return 0;
|
||||
compare = compareFamilies(nextIndexedCell, nextIndexedCell.getFamilyOffset(),
|
||||
nextIndexedCell.getFamilyLength(), currentCell, currentCell.getFamilyOffset(),
|
||||
flen);
|
||||
if (compare != 0) {
|
||||
return compare;
|
||||
}
|
||||
if (colHint == null) {
|
||||
compare = compareQualifiers(nextIndexedCell, nextIndexedCell.getQualifierOffset(),
|
||||
nextIndexedCell.getQualifierLength(), currentCell, currentCell.getQualifierOffset(),
|
||||
qualLen);
|
||||
} else {
|
||||
compare = compareQualifiers(nextIndexedCell, nextIndexedCell.getQualifierOffset(),
|
||||
nextIndexedCell.getQualifierLength(), colHint, coff, clen);
|
||||
}
|
||||
if (compare != 0) {
|
||||
return compare;
|
||||
}
|
||||
// Next compare timestamps.
|
||||
compare = compareTimestamps(nextIndexedCell.getTimestamp(), ts);
|
||||
if (compare != 0) {
|
||||
return compare;
|
||||
}
|
||||
|
||||
int hash = calculateHashForKeyValue(cell);
|
||||
return hash;
|
||||
// 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 & nextIndexedCell.getTypeByte());
|
||||
}
|
||||
|
||||
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 hash = 31 * rowHash + familyHash;
|
||||
hash = 31 * hash + qualifierHash;
|
||||
hash = 31 * hash + (int)cell.getTimestamp();
|
||||
hash = 31 * hash + cell.getTypeByte();
|
||||
return hash;
|
||||
}
|
||||
|
||||
|
||||
/******************** lengths *************************/
|
||||
|
||||
public static boolean areKeyLengthsEqual(Cell a, Cell b) {
|
||||
return a.getRowLength() == b.getRowLength()
|
||||
&& a.getFamilyLength() == b.getFamilyLength()
|
||||
&& a.getQualifierLength() == b.getQualifierLength();
|
||||
}
|
||||
|
||||
public static boolean areRowLengthsEqual(Cell a, Cell b) {
|
||||
return a.getRowLength() == b.getRowLength();
|
||||
}
|
||||
|
||||
|
||||
/*********************common prefixes*************************/
|
||||
|
||||
private static int compare(byte[] left, int leftOffset, int leftLength, byte[] right,
|
||||
int rightOffset, int rightLength) {
|
||||
return Bytes.compareTo(left, leftOffset, leftLength, right, rightOffset, rightLength);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
/***************** special cases ****************************/
|
||||
/**
|
||||
* special case for KeyValue.equals
|
||||
* 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.
|
||||
* @param ltimestamp
|
||||
* @param rtimestamp
|
||||
* @return 1 if left timestamp > right timestamp
|
||||
* -1 if left timestamp < right timestamp
|
||||
* 0 if both timestamps are equal
|
||||
*/
|
||||
public static boolean equalsIgnoreMvccVersion(Cell a, Cell b){
|
||||
return 0 == compareStaticIgnoreMvccVersion(a, b);
|
||||
}
|
||||
|
||||
private static int compareStaticIgnoreMvccVersion(Cell a, Cell b) {
|
||||
// row
|
||||
int c = compareRows(a, b);
|
||||
if (c != 0) return c;
|
||||
|
||||
// family
|
||||
c = compareColumns(a, b);
|
||||
if (c != 0) return c;
|
||||
|
||||
// timestamp: later sorts first
|
||||
c = compareTimestamps(a, b);
|
||||
if (c != 0) return c;
|
||||
|
||||
//type
|
||||
c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
|
||||
return c;
|
||||
}
|
||||
|
||||
private 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.
|
||||
public static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
|
||||
if (ltimestamp < rtimestamp) {
|
||||
return 1;
|
||||
} else if (ltimestamp > rtimestamp) {
|
||||
|
@ -370,7 +614,7 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Counter part for the KeyValue.RowOnlyComparator
|
||||
* Comparator that compares row component only of a Cell
|
||||
*/
|
||||
public static class RowComparator extends CellComparator {
|
||||
@Override
|
||||
|
@ -380,119 +624,61 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Try to return a Cell that falls between <code>left</code> and <code>right</code> but that is
|
||||
* shorter; i.e. takes up less space. This trick is used building HFile block index.
|
||||
* Its an optimization. It does not always work. In this case we'll just return the
|
||||
* <code>right</code> cell.
|
||||
* @param comparator Comparator to use.
|
||||
* @param left
|
||||
* @param right
|
||||
* @return A cell that sorts between <code>left</code> and <code>right</code>.
|
||||
* A {@link CellComparator} for <code>hbase:meta</code> catalog table
|
||||
* {@link KeyValue}s.
|
||||
*/
|
||||
public static Cell getMidpoint(final KeyValue.KVComparator comparator, final Cell left,
|
||||
final Cell right) {
|
||||
// TODO: Redo so only a single pass over the arrays rather than one to compare and then a
|
||||
// second composing midpoint.
|
||||
if (right == null) {
|
||||
throw new IllegalArgumentException("right cell can not be null");
|
||||
}
|
||||
if (left == null) {
|
||||
return right;
|
||||
}
|
||||
// If Cells from meta table, don't mess around. meta table Cells have schema
|
||||
// (table,startrow,hash) so can't be treated as plain byte arrays. Just skip out without
|
||||
// trying to do this optimization.
|
||||
if (comparator != null && comparator instanceof KeyValue.MetaComparator) {
|
||||
return right;
|
||||
}
|
||||
int diff = compareRows(left, right);
|
||||
if (diff > 0) {
|
||||
throw new IllegalArgumentException("Left row sorts after right row; left=" +
|
||||
CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
|
||||
}
|
||||
if (diff < 0) {
|
||||
// Left row is < right row.
|
||||
byte [] midRow = getMinimumMidpointArray(left.getRowArray(), left.getRowOffset(),
|
||||
left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
// If midRow is null, just return 'right'. Can't do optimization.
|
||||
if (midRow == null) return right;
|
||||
return CellUtil.createCell(midRow);
|
||||
}
|
||||
// Rows are same. Compare on families.
|
||||
diff = compareFamilies(left, right);
|
||||
if (diff > 0) {
|
||||
throw new IllegalArgumentException("Left family sorts after right family; left=" +
|
||||
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.
|
||||
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);
|
||||
}
|
||||
// Families are same. Compare on qualifiers.
|
||||
diff = compareQualifiers(left, right);
|
||||
if (diff > 0) {
|
||||
throw new IllegalArgumentException("Left qualifier sorts after right qualifier; left=" +
|
||||
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.
|
||||
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);
|
||||
}
|
||||
// No opportunity for optimization. Just return right key.
|
||||
return right;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param leftArray
|
||||
* @param leftOffset
|
||||
* @param leftLength
|
||||
* @param rightArray
|
||||
* @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) {
|
||||
// rows are different
|
||||
int minLength = leftLength < rightLength ? leftLength : rightLength;
|
||||
short diffIdx = 0;
|
||||
while (diffIdx < minLength &&
|
||||
leftArray[leftOffset + diffIdx] == rightArray[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);
|
||||
} else {
|
||||
int diffByte = leftArray[leftOffset + diffIdx];
|
||||
if ((0xff & diffByte) < 0xff && (diffByte + 1) < (rightArray[rightOffset + diffIdx] & 0xff)) {
|
||||
minimumMidpointArray = new byte[diffIdx + 1];
|
||||
System.arraycopy(leftArray, leftOffset, minimumMidpointArray, 0, diffIdx);
|
||||
minimumMidpointArray[diffIdx] = (byte) (diffByte + 1);
|
||||
public static class MetaCellComparator extends CellComparator {
|
||||
|
||||
@Override
|
||||
public int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
|
||||
int rlength) {
|
||||
int leftDelimiter = Bytes.searchDelimiterIndex(left, loffset, llength, HConstants.DELIMITER);
|
||||
int rightDelimiter = Bytes
|
||||
.searchDelimiterIndex(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 {
|
||||
minimumMidpointArray = new byte[diffIdx + 1];
|
||||
System.arraycopy(rightArray, rightOffset, minimumMidpointArray, 0, diffIdx + 1);
|
||||
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 = Bytes.searchDelimiterIndexInReverse(left, leftDelimiter, llength
|
||||
- (leftDelimiter - loffset), HConstants.DELIMITER);
|
||||
int rightFarDelimiter = Bytes.searchDelimiterIndexInReverse(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;
|
||||
}
|
||||
return minimumMidpointArray;
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ import java.util.List;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.NavigableMap;
|
||||
|
||||
import org.apache.hadoop.hbase.CellComparator.MetaCellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -379,8 +380,10 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
public static boolean matchingRow(final Cell left, final byte[] buf) {
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, 0,
|
||||
buf.length);
|
||||
if (buf == null) {
|
||||
return left.getQualifierLength() == 0;
|
||||
}
|
||||
return matchingRow(left, buf, 0, buf.length);
|
||||
}
|
||||
|
||||
public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
|
||||
|
@ -395,8 +398,10 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
public static boolean matchingFamily(final Cell left, final byte[] buf) {
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
|
||||
0, buf.length);
|
||||
if (buf == null) {
|
||||
return left.getFamilyLength() == 0;
|
||||
}
|
||||
return matchingFamily(left, buf, 0, buf.length);
|
||||
}
|
||||
|
||||
public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
|
||||
|
@ -411,14 +416,29 @@ public final class CellUtil {
|
|||
right.getQualifierLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds if the qualifier part of the cell and the KV serialized
|
||||
* byte[] are equal
|
||||
* @param left
|
||||
* @param buf the serialized keyvalue format byte[]
|
||||
* @return true if the qualifier matches, false otherwise
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds if the qualifier part of the cell and the KV serialized
|
||||
* byte[] are equal
|
||||
* @param left
|
||||
* @param buf the serialized keyvalue format byte[]
|
||||
* @param offset the offset of the qualifier in the byte[]
|
||||
* @param length the length of the qualifier in the byte[]
|
||||
* @return true if the qualifier matches, false otherwise
|
||||
*/
|
||||
public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (buf == null) {
|
||||
|
@ -902,4 +922,118 @@ public final class CellUtil {
|
|||
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/***************** special cases ****************************/
|
||||
|
||||
/**
|
||||
* special case for Cell.equals
|
||||
*/
|
||||
public static boolean equalsIgnoreMvccVersion(Cell a, Cell b) {
|
||||
// row
|
||||
boolean res = matchingRow(a, b);
|
||||
if (!res)
|
||||
return res;
|
||||
|
||||
// family
|
||||
res = matchingColumn(a, b);
|
||||
if (!res)
|
||||
return res;
|
||||
|
||||
// timestamp: later sorts first
|
||||
if (!matchingTimestamp(a, b))
|
||||
return false;
|
||||
|
||||
// type
|
||||
int c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
|
||||
if (c != 0)
|
||||
return false;
|
||||
else return true;
|
||||
}
|
||||
|
||||
/**************** equals ****************************/
|
||||
|
||||
public static boolean equals(Cell a, Cell b) {
|
||||
return matchingRow(a, b) && matchingFamily(a, b) && matchingQualifier(a, b)
|
||||
&& matchingTimestamp(a, b) && matchingType(a, b);
|
||||
}
|
||||
|
||||
public static boolean matchingTimestamp(Cell a, Cell b) {
|
||||
return CellComparator.compareTimestamps(a.getTimestamp(), b.getTimestamp()) == 0;
|
||||
}
|
||||
|
||||
public static boolean matchingType(Cell a, Cell b) {
|
||||
return a.getTypeByte() == b.getTypeByte();
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the row of two keyvalues for equality
|
||||
*
|
||||
* @param left
|
||||
* @param right
|
||||
* @return True if rows match.
|
||||
*/
|
||||
public static boolean matchingRows(final Cell left, final Cell right) {
|
||||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
return matchingRows(left, lrowlength, right, rrowlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param left
|
||||
* @param lrowlength
|
||||
* @param right
|
||||
* @param rrowlength
|
||||
* @return True if rows match.
|
||||
*/
|
||||
private static 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.
|
||||
*/
|
||||
private static 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the row and column of two keyvalues for equality
|
||||
*
|
||||
* @param left
|
||||
* @param right
|
||||
* @return True if same row and column.
|
||||
*/
|
||||
public static boolean matchingRowColumn(final Cell left, final Cell right) {
|
||||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
|
||||
if ((lrowlength + left.getFamilyLength() + left.getQualifierLength()) != (rrowlength
|
||||
+ right.getFamilyLength() + right.getQualifierLength())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!matchingRows(left, lrowlength, right, rrowlength)) {
|
||||
return false;
|
||||
}
|
||||
return matchingColumn(left, right);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -95,17 +95,23 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
/**
|
||||
* Comparator for plain key/values; i.e. non-catalog table key/values. Works on Key portion
|
||||
* of KeyValue only.
|
||||
* @deprecated Use {@link CellComparator#COMPARATOR} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public static final KVComparator COMPARATOR = new KVComparator();
|
||||
/**
|
||||
* A {@link KVComparator} for <code>hbase:meta</code> catalog table
|
||||
* {@link KeyValue}s.
|
||||
* @deprecated Use {@link CellComparator#META_COMPARATOR} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public static final KVComparator META_COMPARATOR = new MetaComparator();
|
||||
|
||||
/**
|
||||
* Needed for Bloom Filters.
|
||||
* * @deprecated Use {@link Bytes#BYTES_RAWCOMPARATOR} instead
|
||||
*/
|
||||
@Deprecated
|
||||
public static final KVComparator RAW_COMPARATOR = new RawBytesComparator();
|
||||
|
||||
/** Size of the key length field in bytes*/
|
||||
|
@ -1061,7 +1067,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
if (!(other instanceof Cell)) {
|
||||
return false;
|
||||
}
|
||||
return CellComparator.equals(this, (Cell)other);
|
||||
return CellUtil.equals(this, (Cell)other);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1069,7 +1075,23 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return CellComparator.hashCodeIgnoreMvcc(this);
|
||||
return calculateHashForKey(this);
|
||||
}
|
||||
|
||||
private int calculateHashForKey(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 hash = 31 * rowHash + familyHash;
|
||||
hash = 31 * hash + qualifierHash;
|
||||
hash = 31 * hash + (int) cell.getTimestamp();
|
||||
hash = 31 * hash + cell.getTypeByte();
|
||||
return hash;
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------
|
||||
|
@ -1714,7 +1736,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
/**
|
||||
* A {@link KVComparator} for <code>hbase:meta</code> catalog table
|
||||
* {@link KeyValue}s.
|
||||
* @deprecated : {@link CellComparator#META_COMPARATOR} to be used
|
||||
*/
|
||||
@Deprecated
|
||||
public static class MetaComparator extends KVComparator {
|
||||
/**
|
||||
* Compare key portion of a {@link KeyValue} for keys in <code>hbase:meta</code>
|
||||
|
@ -1722,11 +1746,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
*/
|
||||
@Override
|
||||
public int compare(final Cell left, final Cell right) {
|
||||
int c = compareRowKey(left, right);
|
||||
if (c != 0) {
|
||||
return c;
|
||||
}
|
||||
return CellComparator.compareWithoutRow(left, right);
|
||||
return CellComparator.META_COMPARATOR.compareKeyIgnoresMvcc(left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1831,7 +1851,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
* 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 CellComparator}.
|
||||
*/
|
||||
@Deprecated
|
||||
public static class KVComparator implements RawComparator<Cell>, SamePrefixComparator<byte[]> {
|
||||
|
||||
/**
|
||||
|
@ -1857,7 +1879,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
* @return 0 if equal, <0 if left smaller, >0 if right smaller
|
||||
*/
|
||||
protected int compareRowKey(final Cell left, final Cell right) {
|
||||
return CellComparator.compareRows(left, right);
|
||||
return CellComparator.COMPARATOR.compareRows(left, right);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1946,7 +1968,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
}
|
||||
|
||||
public int compareOnlyKeyPortion(Cell left, Cell right) {
|
||||
return CellComparator.compare(left, right, true);
|
||||
return CellComparator.COMPARATOR.compareKeyIgnoresMvcc(left, right);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1955,7 +1977,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
*/
|
||||
@Override
|
||||
public int compare(final Cell left, final Cell right) {
|
||||
int compare = CellComparator.compare(left, right, false);
|
||||
int compare = CellComparator.COMPARATOR.compare(left, right);
|
||||
return compare;
|
||||
}
|
||||
|
||||
|
@ -2269,8 +2291,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
* @param leftKey
|
||||
* @param rightKey
|
||||
* @return 0 if equal, <0 if left smaller, >0 if right smaller
|
||||
* @deprecated Since 0.99.2; Use
|
||||
* {@link CellComparator#getMidpoint(KeyValue.KVComparator, Cell, Cell) instead}
|
||||
* @deprecated Since 0.99.2;
|
||||
*/
|
||||
@Deprecated
|
||||
public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
|
||||
|
@ -2556,8 +2577,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
|
|||
}
|
||||
|
||||
/**
|
||||
* This is a TEST only Comparator used in TestSeekTo and TestReseekTo.
|
||||
* @deprecated Not to be used for any comparsions
|
||||
*/
|
||||
@Deprecated
|
||||
public static class RawBytesComparator extends KVComparator {
|
||||
/**
|
||||
* The HFileV2 file format's trailer contains this class name. We reinterpret this and
|
||||
|
|
|
@ -85,7 +85,7 @@ public class KeyValueTestUtil {
|
|||
for (Cell kv1 : kvCollection1) {
|
||||
boolean found = false;
|
||||
for (Cell kv2 : kvCollection2) {
|
||||
if (CellComparator.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
|
||||
if (CellUtil.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
|
||||
}
|
||||
if (!found) return false;
|
||||
}
|
||||
|
|
|
@ -18,7 +18,10 @@
|
|||
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
|
@ -31,6 +34,7 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.IterableUtils;
|
||||
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
|
@ -541,6 +545,122 @@ public class KeyValueUtil {
|
|||
});
|
||||
return new ArrayList<KeyValue>(lazyList);
|
||||
}
|
||||
/**
|
||||
* Write out a KeyValue in the manner in which we used to when KeyValue was a
|
||||
* Writable.
|
||||
*
|
||||
* @param kv
|
||||
* @param out
|
||||
* @return Length written on stream
|
||||
* @throws IOException
|
||||
* @see #create(DataInput) for the inverse function
|
||||
*/
|
||||
public static long write(final KeyValue kv, final DataOutput out) throws IOException {
|
||||
// This is how the old Writables write used to serialize KVs. Need to figure
|
||||
// way to make it
|
||||
// work for all implementations.
|
||||
int length = kv.getLength();
|
||||
out.writeInt(length);
|
||||
out.write(kv.getBuffer(), kv.getOffset(), length);
|
||||
return length + Bytes.SIZEOF_INT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a KeyValue reading from the raw InputStream. Named
|
||||
* <code>iscreate</code> so doesn't clash with {@link #create(DataInput)}
|
||||
*
|
||||
* @param in
|
||||
* @return Created KeyValue OR if we find a length of zero, we will return
|
||||
* null which can be useful marking a stream as done.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static KeyValue iscreate(final InputStream in) throws IOException {
|
||||
byte[] intBytes = new byte[Bytes.SIZEOF_INT];
|
||||
int bytesRead = 0;
|
||||
while (bytesRead < intBytes.length) {
|
||||
int n = in.read(intBytes, bytesRead, intBytes.length - bytesRead);
|
||||
if (n < 0) {
|
||||
if (bytesRead == 0)
|
||||
return null; // EOF at start is ok
|
||||
throw new IOException("Failed read of int, read " + bytesRead + " bytes");
|
||||
}
|
||||
bytesRead += n;
|
||||
}
|
||||
// TODO: perhaps some sanity check is needed here.
|
||||
byte[] bytes = new byte[Bytes.toInt(intBytes)];
|
||||
IOUtils.readFully(in, bytes, 0, bytes.length);
|
||||
return new KeyValue(bytes, 0, bytes.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param b
|
||||
* @return A KeyValue made of a byte array that holds the key-only part.
|
||||
* Needed to convert hfile index members to KeyValues.
|
||||
*/
|
||||
public static KeyValue createKeyValueFromKey(final byte[] b) {
|
||||
return createKeyValueFromKey(b, 0, b.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bb
|
||||
* @return A KeyValue made of a byte buffer that holds the key-only part.
|
||||
* Needed to convert hfile index members to KeyValues.
|
||||
*/
|
||||
public static KeyValue createKeyValueFromKey(final ByteBuffer bb) {
|
||||
return createKeyValueFromKey(bb.array(), bb.arrayOffset(), bb.limit());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param b
|
||||
* @param o
|
||||
* @param l
|
||||
* @return A KeyValue made of a byte array that holds the key-only part.
|
||||
* Needed to convert hfile index members to KeyValues.
|
||||
*/
|
||||
public static KeyValue createKeyValueFromKey(final byte[] b, final int o, final int l) {
|
||||
byte[] newb = new byte[l + KeyValue.ROW_OFFSET];
|
||||
System.arraycopy(b, o, newb, KeyValue.ROW_OFFSET, l);
|
||||
Bytes.putInt(newb, 0, l);
|
||||
Bytes.putInt(newb, Bytes.SIZEOF_INT, 0);
|
||||
return new KeyValue(newb);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param in
|
||||
* Where to read bytes from. Creates a byte array to hold the
|
||||
* KeyValue backing bytes copied from the steam.
|
||||
* @return KeyValue created by deserializing from <code>in</code> OR if we
|
||||
* find a length of zero, we will return null which can be useful
|
||||
* marking a stream as done.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static KeyValue create(final DataInput in) throws IOException {
|
||||
return create(in.readInt(), in);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a KeyValue reading <code>length</code> from <code>in</code>
|
||||
*
|
||||
* @param length
|
||||
* @param in
|
||||
* @return Created KeyValue OR if we find a length of zero, we will return
|
||||
* null which can be useful marking a stream as done.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static KeyValue create(int length, final DataInput in) throws IOException {
|
||||
|
||||
if (length <= 0) {
|
||||
if (length == 0)
|
||||
return null;
|
||||
throw new IOException("Failed read " + length + " bytes, stream corrupt?");
|
||||
}
|
||||
|
||||
// This is how the old Writables.readFrom used to deserialize. Didn't even
|
||||
// vint.
|
||||
byte[] bytes = new byte[length];
|
||||
in.readFully(bytes);
|
||||
return new KeyValue(bytes, 0, length);
|
||||
}
|
||||
|
||||
public static void oswrite(final Cell cell, final OutputStream out, final boolean withTags)
|
||||
throws IOException {
|
||||
|
@ -576,7 +696,8 @@ public class KeyValueUtil {
|
|||
// write tags if we have to
|
||||
if (withTags && tlen > 0) {
|
||||
// 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));
|
||||
|
|
|
@ -23,10 +23,10 @@ import java.util.Arrays;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArraySet;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
|
||||
/**
|
||||
* Immutable POJO class for representing a table name.
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.OutputStream;
|
|||
|
||||
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;
|
||||
|
||||
|
@ -65,7 +64,7 @@ public class KeyValueCodec implements Codec {
|
|||
}
|
||||
|
||||
protected Cell parseCell() throws IOException {
|
||||
return KeyValue.iscreate(in);
|
||||
return KeyValueUtil.iscreate(in);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.OutputStream;
|
|||
|
||||
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;
|
||||
|
||||
|
@ -71,7 +70,7 @@ public class KeyValueCodecWithTags implements Codec {
|
|||
}
|
||||
|
||||
protected Cell parseCell() throws IOException {
|
||||
return KeyValue.iscreate(in);
|
||||
return KeyValueUtil.iscreate(in);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.SettableSequenceId;
|
||||
|
@ -79,6 +77,28 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
return internalDecodeKeyValues(source, 0, 0, decodingCtx);
|
||||
}
|
||||
|
||||
/********************* common prefixes *************************/
|
||||
// Having this as static is fine but if META is having DBE then we should
|
||||
// change this.
|
||||
public static int compareCommonRowPrefix(Cell left, Cell right, int rowCommonPrefix) {
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset() + rowCommonPrefix,
|
||||
left.getRowLength() - rowCommonPrefix, right.getRowArray(), right.getRowOffset()
|
||||
+ rowCommonPrefix, right.getRowLength() - rowCommonPrefix);
|
||||
}
|
||||
|
||||
public static int compareCommonFamilyPrefix(Cell left, Cell right, int familyCommonPrefix) {
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset() + familyCommonPrefix,
|
||||
left.getFamilyLength() - familyCommonPrefix, right.getFamilyArray(),
|
||||
right.getFamilyOffset() + familyCommonPrefix, right.getFamilyLength() - familyCommonPrefix);
|
||||
}
|
||||
|
||||
public static int compareCommonQualifierPrefix(Cell left, Cell right, int qualCommonPrefix) {
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset() + qualCommonPrefix,
|
||||
left.getQualifierLength() - qualCommonPrefix, right.getQualifierArray(),
|
||||
right.getQualifierOffset() + qualCommonPrefix, right.getQualifierLength()
|
||||
- qualCommonPrefix);
|
||||
}
|
||||
|
||||
protected static class SeekerState implements Cell {
|
||||
protected ByteBuffer currentBuffer;
|
||||
protected TagCompressionContext tagCompressionContext;
|
||||
|
@ -520,17 +540,15 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
BufferedEncodedSeeker<STATE extends SeekerState>
|
||||
implements EncodedSeeker {
|
||||
protected HFileBlockDecodingContext decodingCtx;
|
||||
protected final KVComparator comparator;
|
||||
protected final SamePrefixComparator<byte[]> samePrefixComparator;
|
||||
protected final CellComparator comparator;
|
||||
protected ByteBuffer currentBuffer;
|
||||
protected STATE current = createSeekerState(); // always valid
|
||||
protected STATE previous = createSeekerState(); // may not be valid
|
||||
protected TagCompressionContext tagCompressionContext = null;
|
||||
|
||||
public BufferedEncodedSeeker(KVComparator comparator,
|
||||
public BufferedEncodedSeeker(CellComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx) {
|
||||
this.comparator = comparator;
|
||||
this.samePrefixComparator = comparator;
|
||||
this.decodingCtx = decodingCtx;
|
||||
if (decodingCtx.getHFileContext().isCompressTags()) {
|
||||
try {
|
||||
|
@ -540,7 +558,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected boolean includesMvcc() {
|
||||
return this.decodingCtx.getHFileContext().isIncludesMvcc();
|
||||
}
|
||||
|
@ -550,14 +568,11 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
return comparator.compareFlatKey(key, offset, length,
|
||||
current.keyBuffer, 0, current.keyLength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, Cell key) {
|
||||
return comparator.compareOnlyKeyPortion(key,
|
||||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
// TODO BufferedEncodedSeeker, instance will be used by single thread alone. So we can
|
||||
// have one KeyValue.KeyOnlyKeyValue instance as instance variable and reuse here and in
|
||||
// seekToKeyInBlock
|
||||
return comparator.compareKeyIgnoresMvcc(key,
|
||||
new KeyValue.KeyOnlyKeyValue(current.keyBuffer, 0, current.keyLength));
|
||||
}
|
||||
|
||||
|
@ -685,7 +700,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
KeyValue.KeyOnlyKeyValue currentCell = new KeyValue.KeyOnlyKeyValue();
|
||||
do {
|
||||
int comp;
|
||||
if (samePrefixComparator != null) {
|
||||
if (comparator != null) {
|
||||
currentCell.setKey(current.keyBuffer, 0, current.keyLength);
|
||||
if (current.lastCommonPrefix != 0) {
|
||||
// The KV format has row key length also in the byte array. The
|
||||
|
@ -698,9 +713,10 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
if (current.lastCommonPrefix <= 2) {
|
||||
rowCommonPrefix = 0;
|
||||
}
|
||||
rowCommonPrefix += CellComparator.findCommonPrefixInRowPart(seekCell, currentCell,
|
||||
rowCommonPrefix += findCommonPrefixInRowPart(seekCell, currentCell,
|
||||
rowCommonPrefix);
|
||||
comp = compareCommonRowPrefix(seekCell, currentCell,
|
||||
rowCommonPrefix);
|
||||
comp = CellComparator.compareCommonRowPrefix(seekCell, currentCell, rowCommonPrefix);
|
||||
if (comp == 0) {
|
||||
comp = compareTypeBytes(seekCell, currentCell);
|
||||
if (comp == 0) {
|
||||
|
@ -709,9 +725,9 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
0,
|
||||
Math.min(familyCommonPrefix,
|
||||
current.lastCommonPrefix - (3 + currentCell.getRowLength())));
|
||||
familyCommonPrefix += CellComparator.findCommonPrefixInFamilyPart(seekCell,
|
||||
familyCommonPrefix += findCommonPrefixInFamilyPart(seekCell,
|
||||
currentCell, familyCommonPrefix);
|
||||
comp = CellComparator.compareCommonFamilyPrefix(seekCell, currentCell,
|
||||
comp = compareCommonFamilyPrefix(seekCell, currentCell,
|
||||
familyCommonPrefix);
|
||||
if (comp == 0) {
|
||||
// subtract the rowkey fixed length and the family key fixed
|
||||
|
@ -722,9 +738,9 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
qualCommonPrefix,
|
||||
current.lastCommonPrefix
|
||||
- (3 + currentCell.getRowLength() + currentCell.getFamilyLength())));
|
||||
qualCommonPrefix += CellComparator.findCommonPrefixInQualifierPart(seekCell,
|
||||
qualCommonPrefix += findCommonPrefixInQualifierPart(seekCell,
|
||||
currentCell, qualCommonPrefix);
|
||||
comp = CellComparator.compareCommonQualifierPrefix(seekCell, currentCell,
|
||||
comp = compareCommonQualifierPrefix(seekCell, currentCell,
|
||||
qualCommonPrefix);
|
||||
if (comp == 0) {
|
||||
comp = CellComparator.compareTimestamps(seekCell, currentCell);
|
||||
|
@ -745,7 +761,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
}
|
||||
} else {
|
||||
Cell r = new KeyValue.KeyOnlyKeyValue(current.keyBuffer, 0, current.keyLength);
|
||||
comp = comparator.compareOnlyKeyPortion(seekCell, r);
|
||||
comp = comparator.compareKeyIgnoresMvcc(seekCell, r);
|
||||
}
|
||||
if (comp == 0) { // exact match
|
||||
if (seekBefore) {
|
||||
|
@ -798,6 +814,27 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
return 0;
|
||||
}
|
||||
|
||||
private static int findCommonPrefixInRowPart(Cell left, Cell right, int rowCommonPrefix) {
|
||||
return Bytes.findCommonPrefix(left.getRowArray(), right.getRowArray(), left.getRowLength()
|
||||
- rowCommonPrefix, right.getRowLength() - rowCommonPrefix, left.getRowOffset()
|
||||
+ rowCommonPrefix, right.getRowOffset() + rowCommonPrefix);
|
||||
}
|
||||
|
||||
private static int findCommonPrefixInFamilyPart(Cell left, Cell right, int familyCommonPrefix) {
|
||||
return Bytes
|
||||
.findCommonPrefix(left.getFamilyArray(), right.getFamilyArray(), left.getFamilyLength()
|
||||
- familyCommonPrefix, right.getFamilyLength() - familyCommonPrefix,
|
||||
left.getFamilyOffset() + familyCommonPrefix, right.getFamilyOffset()
|
||||
+ familyCommonPrefix);
|
||||
}
|
||||
|
||||
private static int findCommonPrefixInQualifierPart(Cell left, Cell right,
|
||||
int qualifierCommonPrefix) {
|
||||
return Bytes.findCommonPrefix(left.getQualifierArray(), right.getQualifierArray(),
|
||||
left.getQualifierLength() - qualifierCommonPrefix, right.getQualifierLength()
|
||||
- qualifierCommonPrefix, left.getQualifierOffset() + qualifierCommonPrefix,
|
||||
right.getQualifierOffset() + qualifierCommonPrefix);
|
||||
}
|
||||
|
||||
private void moveToPrevious() {
|
||||
if (!previous.isValid()) {
|
||||
|
|
|
@ -22,9 +22,9 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.util.ByteBufferUtils;
|
||||
|
@ -81,7 +81,7 @@ public class CopyKeyDataBlockEncoder extends BufferedDataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public EncodedSeeker createSeeker(KVComparator comparator,
|
||||
public EncodedSeeker createSeeker(CellComparator comparator,
|
||||
final HFileBlockDecodingContext decodingCtx) {
|
||||
return new BufferedEncodedSeeker<SeekerState>(comparator, decodingCtx) {
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
|
||||
|
@ -98,7 +98,7 @@ public interface DataBlockEncoder {
|
|||
* @param decodingCtx
|
||||
* @return A newly created seeker.
|
||||
*/
|
||||
EncodedSeeker createSeeker(KVComparator comparator,
|
||||
EncodedSeeker createSeeker(CellComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx);
|
||||
|
||||
/**
|
||||
|
@ -193,12 +193,8 @@ public interface DataBlockEncoder {
|
|||
* 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
|
||||
*/
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length);
|
||||
|
||||
public int compareKey(KVComparator comparator, Cell key);
|
||||
public int compareKey(CellComparator comparator, Cell key);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,9 +22,9 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.util.ByteBufferUtils;
|
||||
|
@ -372,7 +372,7 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public EncodedSeeker createSeeker(KVComparator comparator,
|
||||
public EncodedSeeker createSeeker(CellComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx) {
|
||||
return new BufferedEncodedSeeker<DiffSeekerState>(comparator, decodingCtx) {
|
||||
private byte[] familyNameWithSize;
|
||||
|
|
|
@ -22,9 +22,9 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.util.ByteBufferUtils;
|
||||
|
@ -392,7 +392,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public EncodedSeeker createSeeker(KVComparator comparator,
|
||||
public EncodedSeeker createSeeker(CellComparator comparator,
|
||||
final HFileBlockDecodingContext decodingCtx) {
|
||||
return new BufferedEncodedSeeker<FastDiffSeekerState>(comparator, decodingCtx) {
|
||||
private void decode(boolean isFirst) {
|
||||
|
|
|
@ -22,9 +22,9 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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.util.ByteBufferUtils;
|
||||
|
@ -196,7 +196,7 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public EncodedSeeker createSeeker(KVComparator comparator,
|
||||
public EncodedSeeker createSeeker(CellComparator comparator,
|
||||
final HFileBlockDecodingContext decodingCtx) {
|
||||
return new BufferedEncodedSeeker<SeekerState>(comparator, decodingCtx) {
|
||||
@Override
|
||||
|
|
|
@ -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;
|
||||
|
@ -2088,8 +2089,9 @@ public class Bytes implements Comparable<Bytes> {
|
|||
* ranging from -(N + 1) to N - 1.
|
||||
*/
|
||||
public static int binarySearch(byte[][] arr, byte[] key, int offset, int length) {
|
||||
return binarySearch(arr, key, offset, length, Bytes.BYTES_RAWCOMPARATOR);
|
||||
return binarySearch(arr, key, offset, length, (CellComparator) null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Binary search for keys in indexes.
|
||||
*
|
||||
|
@ -2109,18 +2111,41 @@ public class Bytes implements Comparable<Bytes> {
|
|||
@Deprecated
|
||||
public static int binarySearch(byte [][]arr, byte []key, int offset,
|
||||
int length, RawComparator<?> comparator) {
|
||||
if(comparator == null) {
|
||||
comparator = Bytes.BYTES_RAWCOMPARATOR;
|
||||
}
|
||||
return binarySearch(arr, key, offset, length, (CellComparator)null);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, byte []key, int offset,
|
||||
int length, CellComparator comparator) {
|
||||
int low = 0;
|
||||
int high = arr.length - 1;
|
||||
|
||||
KeyValue.KeyOnlyKeyValue r = new KeyValue.KeyOnlyKeyValue();
|
||||
r.setKey(key, offset, length);
|
||||
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.
|
||||
int cmp = comparator.compare(key, offset, length,
|
||||
arr[mid], 0, arr[mid].length);
|
||||
int cmp = 0;
|
||||
if (comparator != null) {
|
||||
cmp = comparator.compare(r, arr[mid], 0, arr[mid].length);
|
||||
} else {
|
||||
cmp = Bytes.BYTES_RAWCOMPARATOR.compare(key, offset, length, arr[mid], 0, arr[mid].length);
|
||||
}
|
||||
// key lives above the midpoint
|
||||
if (cmp > 0)
|
||||
low = mid + 1;
|
||||
|
@ -2618,4 +2643,56 @@ public class Bytes implements Comparable<Bytes> {
|
|||
return b;
|
||||
}
|
||||
|
||||
}
|
||||
/**
|
||||
* @param b
|
||||
* @param delimiter
|
||||
* @return Index of delimiter having started from start of <code>b</code> moving rightward.
|
||||
*/
|
||||
public static int searchDelimiterIndex(final byte[] 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 (b[i] == delimiter) {
|
||||
result = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Find index of passed delimiter walking from end of buffer backwards.
|
||||
*
|
||||
* @param b
|
||||
* @param delimiter
|
||||
* @return Index of delimiter
|
||||
*/
|
||||
public static int searchDelimiterIndexInReverse(final byte[] 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[i] == delimiter) {
|
||||
result = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public static int findCommonPrefix(byte[] left, byte[] right, int leftLength, int rightLength,
|
||||
int leftOffset, int rightOffset) {
|
||||
int length = Math.min(leftLength, rightLength);
|
||||
int result = 0;
|
||||
|
||||
while (result < length && left[leftOffset + result] == right[rightOffset + result]) {
|
||||
result++;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -284,7 +285,7 @@ public class RedundantKVGenerator {
|
|||
}
|
||||
}
|
||||
|
||||
Collections.sort(result, KeyValue.COMPARATOR);
|
||||
Collections.sort(result, CellComparator.COMPARATOR);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -17,10 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -30,6 +28,7 @@ import org.junit.experimental.categories.Category;
|
|||
@Category({MiscTests.class, SmallTests.class})
|
||||
public class TestCellComparator {
|
||||
|
||||
private CellComparator comparator = CellComparator.COMPARATOR;
|
||||
byte[] row1 = Bytes.toBytes("row1");
|
||||
byte[] row2 = Bytes.toBytes("row2");
|
||||
byte[] row_1_0 = Bytes.toBytes("row10");
|
||||
|
@ -47,7 +46,7 @@ public class TestCellComparator {
|
|||
public void testCompareCells() {
|
||||
KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
|
||||
KeyValue kv2 = new KeyValue(row2, fam1, qual1, val);
|
||||
assertTrue((CellComparator.compare(kv1, kv2, false)) < 0);
|
||||
assertTrue((comparator.compare(kv1, kv2)) < 0);
|
||||
|
||||
kv1 = new KeyValue(row1, fam2, qual1, val);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, val);
|
||||
|
@ -55,90 +54,37 @@ public class TestCellComparator {
|
|||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, val);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 2l, val);
|
||||
assertTrue((CellComparator.compare(kv1, kv2, false) > 0));
|
||||
assertTrue((comparator.compare(kv1, kv2) > 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
|
||||
assertTrue((CellComparator.compare(kv1, kv2, false) > 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam_1_2, qual1, 1l, Type.Maximum);
|
||||
assertTrue((CellComparator.compareCommonFamilyPrefix(kv1, kv2, 4) < 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row_1_0, fam_1_2, qual1, 1l, Type.Maximum);
|
||||
assertTrue((CellComparator.compareCommonRowPrefix(kv1, kv2, 4) < 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual2, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
|
||||
assertTrue((CellComparator.compareCommonQualifierPrefix(kv1, kv2, 4) > 0));
|
||||
assertTrue((comparator.compare(kv1, kv2) > 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
assertTrue((CellComparator.equals(kv1, kv2)));
|
||||
assertTrue((CellUtil.equals(kv1, kv2)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetShortMidpoint() {
|
||||
KeyValue.KVComparator comparator = new KeyValue.KVComparator();
|
||||
public void testCompareCellWithKey() throws Exception {
|
||||
KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
|
||||
KeyValue kv2 = new KeyValue(row2, fam1, qual1, val);
|
||||
assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) < 0);
|
||||
|
||||
Cell left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
Cell right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
Cell mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) <= 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) <= 0);
|
||||
kv1 = new KeyValue(row1, fam2, qual1, val);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, val);
|
||||
assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("b"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) <= 0);
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, val);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 2l, val);
|
||||
assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("g"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("i"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) <= 0);
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
|
||||
assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("bbbbbbb"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) < 0);
|
||||
assertEquals(1, (int)mid.getRowLength());
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("a"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) <= 0);
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("aaaaaaaa"), Bytes.toBytes("b"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) < 0);
|
||||
assertEquals(2, (int)mid.getFamilyLength());
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("aaaaaaaaa"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) < 0);
|
||||
assertEquals(2, (int)mid.getQualifierLength());
|
||||
|
||||
left = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("a"), Bytes.toBytes("a"), Bytes.toBytes("b"));
|
||||
mid = CellComparator.getMidpoint(comparator, left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) <= 0);
|
||||
assertEquals(1, (int)mid.getQualifierLength());
|
||||
|
||||
// Assert that if meta comparator, it returns the right cell -- i.e. no optimization done.
|
||||
left = CellUtil.createCell(Bytes.toBytes("g"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
right = CellUtil.createCell(Bytes.toBytes("i"), Bytes.toBytes("a"), Bytes.toBytes("a"));
|
||||
mid = CellComparator.getMidpoint(new KeyValue.MetaComparator(), left, right);
|
||||
assertTrue(CellComparator.compare(left, mid, true) < 0);
|
||||
assertTrue(CellComparator.compare(mid, right, true) == 0);
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) == 0);
|
||||
}
|
||||
}
|
|
@ -23,7 +23,6 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -34,8 +33,6 @@ import junit.framework.TestCase;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.MetaComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -109,24 +106,24 @@ public class TestKeyValue extends TestCase {
|
|||
final byte [] qf = Bytes.toBytes("umn");
|
||||
KeyValue aaa = new KeyValue(a, fam, qf, a);
|
||||
KeyValue bbb = new KeyValue(b, fam, qf, b);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(aaa, bbb) < 0);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(bbb, aaa) > 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(aaa, bbb) < 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(bbb, aaa) > 0);
|
||||
// Compare breaks if passed same ByteBuffer as both left and right arguments.
|
||||
assertTrue(KeyValue.COMPARATOR.compare(bbb, bbb) == 0);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(aaa, aaa) == 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(bbb, bbb) == 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(aaa, aaa) == 0);
|
||||
// Do compare with different timestamps.
|
||||
aaa = new KeyValue(a, fam, qf, 1, a);
|
||||
bbb = new KeyValue(a, fam, qf, 2, a);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(aaa, bbb) > 0);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(bbb, aaa) < 0);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(aaa, aaa) == 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(aaa, bbb) > 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(bbb, aaa) < 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(aaa, aaa) == 0);
|
||||
// Do compare with different types. Higher numbered types -- Delete
|
||||
// should sort ahead of lower numbers; i.e. Put
|
||||
aaa = new KeyValue(a, fam, qf, 1, KeyValue.Type.Delete, a);
|
||||
bbb = new KeyValue(a, fam, qf, 1, a);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(aaa, bbb) < 0);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(bbb, aaa) > 0);
|
||||
assertTrue(KeyValue.COMPARATOR.compare(aaa, aaa) == 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(aaa, bbb) < 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(bbb, aaa) > 0);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(aaa, aaa) == 0);
|
||||
}
|
||||
|
||||
public void testMoreComparisons() throws Exception {
|
||||
|
@ -137,7 +134,7 @@ public class TestKeyValue extends TestCase {
|
|||
Bytes.toBytes("TestScanMultipleVersions,row_0500,1236020145502"), now);
|
||||
KeyValue bbb = new KeyValue(
|
||||
Bytes.toBytes("TestScanMultipleVersions,,99999999999999"), now);
|
||||
KVComparator c = new KeyValue.MetaComparator();
|
||||
CellComparator c = CellComparator.META_COMPARATOR;
|
||||
assertTrue(c.compare(bbb, aaa) < 0);
|
||||
|
||||
KeyValue aaaa = new KeyValue(Bytes.toBytes("TestScanMultipleVersions,,1236023996656"),
|
||||
|
@ -152,13 +149,13 @@ public class TestKeyValue extends TestCase {
|
|||
Bytes.toBytes("info"), Bytes.toBytes("regioninfo"), 1236034574912L,
|
||||
(byte[])null);
|
||||
assertTrue(c.compare(x, y) < 0);
|
||||
comparisons(new KeyValue.MetaComparator());
|
||||
comparisons(new KeyValue.KVComparator());
|
||||
metacomparisons(new KeyValue.MetaComparator());
|
||||
comparisons(CellComparator.META_COMPARATOR);
|
||||
comparisons(CellComparator.COMPARATOR);
|
||||
metacomparisons(CellComparator.META_COMPARATOR);
|
||||
}
|
||||
|
||||
public void testMetaComparatorTableKeysWithCommaOk() {
|
||||
MetaComparator c = new KeyValue.MetaComparator();
|
||||
CellComparator c = CellComparator.META_COMPARATOR;
|
||||
long now = System.currentTimeMillis();
|
||||
// meta keys values are not quite right. A users can enter illegal values
|
||||
// from shell when scanning meta.
|
||||
|
@ -179,17 +176,17 @@ public class TestKeyValue extends TestCase {
|
|||
Bytes.toBytes("fam"), Bytes.toBytes(""), Long.MAX_VALUE, (byte[])null);
|
||||
KeyValue rowB = new KeyValue(Bytes.toBytes("testtable,www.hbase.org/%20,99999"),
|
||||
Bytes.toBytes("fam"), Bytes.toBytes(""), Long.MAX_VALUE, (byte[])null);
|
||||
assertTrue(KeyValue.META_COMPARATOR.compare(rowA, rowB) < 0);
|
||||
assertTrue(CellComparator.META_COMPARATOR.compare(rowA, rowB) < 0);
|
||||
|
||||
rowA = new KeyValue(Bytes.toBytes("testtable,,1234"), Bytes.toBytes("fam"),
|
||||
Bytes.toBytes(""), Long.MAX_VALUE, (byte[])null);
|
||||
rowB = new KeyValue(Bytes.toBytes("testtable,$www.hbase.org/,99999"),
|
||||
Bytes.toBytes("fam"), Bytes.toBytes(""), Long.MAX_VALUE, (byte[])null);
|
||||
assertTrue(KeyValue.META_COMPARATOR.compare(rowA, rowB) < 0);
|
||||
assertTrue(CellComparator.META_COMPARATOR.compare(rowA, rowB) < 0);
|
||||
|
||||
}
|
||||
|
||||
private void metacomparisons(final KeyValue.MetaComparator c) {
|
||||
private void metacomparisons(final CellComparator c) {
|
||||
long now = System.currentTimeMillis();
|
||||
assertTrue(c.compare(new KeyValue(
|
||||
Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now),
|
||||
|
@ -206,7 +203,7 @@ public class TestKeyValue extends TestCase {
|
|||
Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now)) > 0);
|
||||
}
|
||||
|
||||
private void comparisons(final KeyValue.KVComparator c) {
|
||||
private void comparisons(final CellComparator c) {
|
||||
long now = System.currentTimeMillis();
|
||||
assertTrue(c.compare(new KeyValue(
|
||||
Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now),
|
||||
|
@ -223,7 +220,7 @@ public class TestKeyValue extends TestCase {
|
|||
}
|
||||
|
||||
public void testBinaryKeys() throws Exception {
|
||||
Set<KeyValue> set = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
|
||||
Set<KeyValue> set = new TreeSet<KeyValue>(CellComparator.COMPARATOR);
|
||||
final byte [] fam = Bytes.toBytes("col");
|
||||
final byte [] qf = Bytes.toBytes("umn");
|
||||
final byte [] nb = new byte[0];
|
||||
|
@ -249,7 +246,7 @@ public class TestKeyValue extends TestCase {
|
|||
}
|
||||
assertTrue(assertion);
|
||||
// Make set with good comparator
|
||||
set = new TreeSet<KeyValue>(new KeyValue.MetaComparator());
|
||||
set = new TreeSet<KeyValue>(CellComparator.META_COMPARATOR);
|
||||
Collections.addAll(set, keys);
|
||||
count = 0;
|
||||
for (KeyValue k: set) {
|
||||
|
@ -271,7 +268,7 @@ public class TestKeyValue extends TestCase {
|
|||
private final byte[] qualA = Bytes.toBytes("qfA");
|
||||
private final byte[] qualB = Bytes.toBytes("qfB");
|
||||
|
||||
private void assertKVLess(KeyValue.KVComparator c,
|
||||
private void assertKVLess(CellComparator c,
|
||||
KeyValue less,
|
||||
KeyValue greater) {
|
||||
int cmp = c.compare(less,greater);
|
||||
|
@ -280,20 +277,16 @@ public class TestKeyValue extends TestCase {
|
|||
assertTrue(cmp > 0);
|
||||
}
|
||||
|
||||
private void assertKVLessWithoutRow(KeyValue.KVComparator c, int common, KeyValue less,
|
||||
private void assertKVLessWithoutRow(CellComparator c, int common, KeyValue less,
|
||||
KeyValue greater) {
|
||||
int cmp = c.compareIgnoringPrefix(common, less.getBuffer(), less.getOffset()
|
||||
+ KeyValue.ROW_OFFSET, less.getKeyLength(), greater.getBuffer(),
|
||||
greater.getOffset() + KeyValue.ROW_OFFSET, greater.getKeyLength());
|
||||
int cmp = c.compare(less, greater);
|
||||
assertTrue(cmp < 0);
|
||||
cmp = c.compareIgnoringPrefix(common, greater.getBuffer(), greater.getOffset()
|
||||
+ KeyValue.ROW_OFFSET, greater.getKeyLength(), less.getBuffer(),
|
||||
less.getOffset() + KeyValue.ROW_OFFSET, less.getKeyLength());
|
||||
cmp = c.compare(greater, less);
|
||||
assertTrue(cmp > 0);
|
||||
}
|
||||
|
||||
public void testCompareWithoutRow() {
|
||||
final KeyValue.KVComparator c = KeyValue.COMPARATOR;
|
||||
final CellComparator c = CellComparator.COMPARATOR;
|
||||
byte[] row = Bytes.toBytes("row");
|
||||
|
||||
byte[] fa = Bytes.toBytes("fa");
|
||||
|
@ -340,7 +333,7 @@ public class TestKeyValue extends TestCase {
|
|||
}
|
||||
|
||||
public void testFirstLastOnRow() {
|
||||
final KVComparator c = KeyValue.COMPARATOR;
|
||||
final CellComparator c = CellComparator.COMPARATOR;
|
||||
long ts = 1;
|
||||
byte[] bufferA = new byte[128];
|
||||
int offsetA = 0;
|
||||
|
@ -414,7 +407,7 @@ public class TestKeyValue extends TestCase {
|
|||
byte[] tmpArr = new byte[initialPadding + endingPadding + keyLen];
|
||||
System.arraycopy(kv.getBuffer(), kv.getKeyOffset(), tmpArr,
|
||||
initialPadding, keyLen);
|
||||
KeyValue kvFromKey = KeyValue.createKeyValueFromKey(tmpArr, initialPadding,
|
||||
KeyValue kvFromKey = KeyValueUtil.createKeyValueFromKey(tmpArr, initialPadding,
|
||||
keyLen);
|
||||
assertEquals(keyLen, kvFromKey.getKeyLength());
|
||||
assertEquals(KeyValue.ROW_OFFSET + keyLen, kvFromKey.getBuffer().length);
|
||||
|
@ -439,82 +432,6 @@ public class TestKeyValue extends TestCase {
|
|||
assertEquals(12345L, time2);
|
||||
}
|
||||
|
||||
/**
|
||||
* See HBASE-7845
|
||||
*/
|
||||
public void testGetShortMidpointKey() {
|
||||
final KVComparator keyComparator = KeyValue.COMPARATOR;
|
||||
//verify that faked shorter rowkey could be generated
|
||||
long ts = 5;
|
||||
KeyValue kv1 = new KeyValue(Bytes.toBytes("the quick brown fox"), family, qualA, ts, Type.Put);
|
||||
KeyValue kv2 = new KeyValue(Bytes.toBytes("the who test text"), family, qualA, ts, Type.Put);
|
||||
byte[] newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compareFlatKey(newKey, kv2.getKey()) < 0);
|
||||
short newRowLength = Bytes.toShort(newKey, 0);
|
||||
byte[] expectedArray = Bytes.toBytes("the r");
|
||||
Bytes.equals(newKey, KeyValue.ROW_LENGTH_SIZE, newRowLength, expectedArray, 0,
|
||||
expectedArray.length);
|
||||
|
||||
//verify: same with "row + family + qualifier", return rightKey directly
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, 5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, 0, Type.Put);
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compareFlatKey(newKey, kv2.getKey()) == 0);
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, -5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, -10, Type.Put);
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compareFlatKey(newKey, kv2.getKey()) == 0);
|
||||
|
||||
// verify: same with row, different with qualifier
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, 5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualB, 5, Type.Put);
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compareFlatKey(newKey, kv2.getKey()) < 0);
|
||||
KeyValue newKeyValue = KeyValue.createKeyValueFromKey(newKey);
|
||||
assertTrue(Arrays.equals(newKeyValue.getFamily(),family));
|
||||
assertTrue(Arrays.equals(newKeyValue.getQualifier(),qualB));
|
||||
assertTrue(newKeyValue.getTimestamp() == HConstants.LATEST_TIMESTAMP);
|
||||
assertTrue(newKeyValue.getTypeByte() == Type.Maximum.getCode());
|
||||
|
||||
//verify metaKeyComparator's getShortMidpointKey output
|
||||
final KVComparator metaKeyComparator = KeyValue.META_COMPARATOR;
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase123"), family, qualA, 5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase234"), family, qualA, 0, Type.Put);
|
||||
newKey = metaKeyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(metaKeyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(metaKeyComparator.compareFlatKey(newKey, kv2.getKey()) == 0);
|
||||
|
||||
//verify common fix scenario
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, ts, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbaseandhdfs"), family, qualA, ts, Type.Put);
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compareFlatKey(newKey, kv2.getKey()) < 0);
|
||||
newRowLength = Bytes.toShort(newKey, 0);
|
||||
expectedArray = Bytes.toBytes("ilovehbasea");
|
||||
Bytes.equals(newKey, KeyValue.ROW_LENGTH_SIZE, newRowLength, expectedArray, 0,
|
||||
expectedArray.length);
|
||||
//verify only 1 offset scenario
|
||||
kv1 = new KeyValue(Bytes.toBytes("100abcdefg"), family, qualA, ts, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("101abcdefg"), family, qualA, ts, Type.Put);
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compareFlatKey(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compareFlatKey(newKey, kv2.getKey()) < 0);
|
||||
newRowLength = Bytes.toShort(newKey, 0);
|
||||
expectedArray = Bytes.toBytes("101");
|
||||
Bytes.equals(newKey, KeyValue.ROW_LENGTH_SIZE, newRowLength, expectedArray, 0,
|
||||
expectedArray.length);
|
||||
}
|
||||
|
||||
public void testKVsWithTags() {
|
||||
byte[] row = Bytes.toBytes("myRow");
|
||||
byte[] cf = Bytes.toBytes("myCF");
|
||||
|
@ -577,7 +494,7 @@ public class TestKeyValue extends TestCase {
|
|||
}
|
||||
|
||||
public void testMetaKeyComparator() {
|
||||
MetaComparator c = new KeyValue.MetaComparator();
|
||||
CellComparator c = CellComparator.META_COMPARATOR;
|
||||
long now = System.currentTimeMillis();
|
||||
|
||||
KeyValue a = new KeyValue(Bytes.toBytes("table1"), now);
|
||||
|
@ -652,9 +569,9 @@ public class TestKeyValue extends TestCase {
|
|||
KeyValueUtil.oswrite(mkvA2, os, true);
|
||||
DataInputStream is = new DataInputStream(new ByteArrayInputStream(
|
||||
byteArrayOutputStream.toByteArray()));
|
||||
KeyValue deSerKV1 = KeyValue.iscreate(is);
|
||||
KeyValue deSerKV1 = KeyValueUtil.iscreate(is);
|
||||
assertTrue(kvA1.equals(deSerKV1));
|
||||
KeyValue deSerKV2 = KeyValue.iscreate(is);
|
||||
KeyValue deSerKV2 = KeyValueUtil.iscreate(is);
|
||||
assertTrue(kvA2.equals(deSerKV2));
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -112,13 +113,13 @@ public class TestCellCodec {
|
|||
Codec.Decoder decoder = codec.getDecoder(dis);
|
||||
assertTrue(decoder.advance());
|
||||
Cell c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv1));
|
||||
assertTrue(CellUtil.equals(c, kv1));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv2));
|
||||
assertTrue(CellUtil.equals(c, kv2));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv3));
|
||||
assertTrue(CellUtil.equals(c, kv3));
|
||||
assertFalse(decoder.advance());
|
||||
dis.close();
|
||||
assertEquals(offset, cis.getCount());
|
||||
|
|
|
@ -29,7 +29,7 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
|
@ -76,7 +76,7 @@ public class TestCellCodecWithTags {
|
|||
Codec.Decoder decoder = codec.getDecoder(dis);
|
||||
assertTrue(decoder.advance());
|
||||
Cell c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, cell1));
|
||||
assertTrue(CellUtil.equals(c, cell1));
|
||||
List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
assertEquals(2, tags.size());
|
||||
Tag tag = tags.get(0);
|
||||
|
@ -87,7 +87,7 @@ public class TestCellCodecWithTags {
|
|||
assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), tag.getValue()));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, cell2));
|
||||
assertTrue(CellUtil.equals(c, cell2));
|
||||
tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
assertEquals(1, tags.size());
|
||||
tag = tags.get(0);
|
||||
|
@ -95,7 +95,7 @@ public class TestCellCodecWithTags {
|
|||
assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), tag.getValue()));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, cell3));
|
||||
assertTrue(CellUtil.equals(c, cell3));
|
||||
tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
assertEquals(3, tags.size());
|
||||
tag = tags.get(0);
|
||||
|
|
|
@ -29,7 +29,7 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
|
@ -76,7 +76,7 @@ public class TestKeyValueCodecWithTags {
|
|||
Codec.Decoder decoder = codec.getDecoder(dis);
|
||||
assertTrue(decoder.advance());
|
||||
Cell c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv1));
|
||||
assertTrue(CellUtil.equals(c, kv1));
|
||||
List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
assertEquals(2, tags.size());
|
||||
Tag tag = tags.get(0);
|
||||
|
@ -87,7 +87,7 @@ public class TestKeyValueCodecWithTags {
|
|||
assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), tag.getValue()));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv2));
|
||||
assertTrue(CellUtil.equals(c, kv2));
|
||||
tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
assertEquals(1, tags.size());
|
||||
tag = tags.get(0);
|
||||
|
@ -95,7 +95,7 @@ public class TestKeyValueCodecWithTags {
|
|||
assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), tag.getValue()));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv3));
|
||||
assertTrue(CellUtil.equals(c, kv3));
|
||||
tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
|
||||
assertEquals(3, tags.size());
|
||||
tag = tags.get(0);
|
||||
|
|
|
@ -285,33 +285,25 @@ public class TestBytes extends TestCase {
|
|||
byte [] key4 = {0};
|
||||
byte [] key5 = {2};
|
||||
|
||||
assertEquals(1, Bytes.binarySearch(arr, key1, 0, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
assertEquals(0, Bytes.binarySearch(arr, key1, 1, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
assertEquals(1, Bytes.binarySearch(arr, key1, 0, 1));
|
||||
assertEquals(0, Bytes.binarySearch(arr, key1, 1, 1));
|
||||
assertEquals(-(2+1), Arrays.binarySearch(arr, key2_2,
|
||||
Bytes.BYTES_COMPARATOR));
|
||||
assertEquals(-(2+1), Bytes.binarySearch(arr, key2, 0, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
assertEquals(4, Bytes.binarySearch(arr, key2, 1, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
assertEquals(2, Bytes.binarySearch(arr, key3, 0, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
assertEquals(5, Bytes.binarySearch(arr, key3, 1, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
assertEquals(-(2+1), Bytes.binarySearch(arr, key2, 0, 1));
|
||||
assertEquals(4, Bytes.binarySearch(arr, key2, 1, 1));
|
||||
assertEquals(2, Bytes.binarySearch(arr, key3, 0, 1));
|
||||
assertEquals(5, Bytes.binarySearch(arr, key3, 1, 1));
|
||||
assertEquals(-1,
|
||||
Bytes.binarySearch(arr, key4, 0, 1, Bytes.BYTES_RAWCOMPARATOR));
|
||||
Bytes.binarySearch(arr, key4, 0, 1));
|
||||
assertEquals(-2,
|
||||
Bytes.binarySearch(arr, key5, 0, 1, Bytes.BYTES_RAWCOMPARATOR));
|
||||
Bytes.binarySearch(arr, key5, 0, 1));
|
||||
|
||||
// Search for values to the left and to the right of each item in the array.
|
||||
for (int i = 0; i < arr.length; ++i) {
|
||||
assertEquals(-(i + 1), Bytes.binarySearch(arr,
|
||||
new byte[] { (byte) (arr[i][0] - 1) }, 0, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
new byte[] { (byte) (arr[i][0] - 1) }, 0, 1));
|
||||
assertEquals(-(i + 2), Bytes.binarySearch(arr,
|
||||
new byte[] { (byte) (arr[i][0] + 1) }, 0, 1,
|
||||
Bytes.BYTES_RAWCOMPARATOR));
|
||||
new byte[] { (byte) (arr[i][0] + 1) }, 0, 1));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -52,6 +53,6 @@ public class TestPBCell {
|
|||
pbr.setPosition(0);
|
||||
decoded = CODEC.decode(pbr);
|
||||
assertEquals(encodedLength, pbr.getPosition());
|
||||
assertTrue(CellComparator.equals(cell, ProtobufUtil.toCell(decoded)));
|
||||
assertTrue(CellUtil.equals(cell, ProtobufUtil.toCell(decoded)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ 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.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.IntegrationTestingUtility;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
|
@ -78,7 +79,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
|
|||
"row10\t1\tc1\tc2\n";
|
||||
|
||||
protected static final Set<KeyValue> simple_expected =
|
||||
new TreeSet<KeyValue>(KeyValue.COMPARATOR) {
|
||||
new TreeSet<KeyValue>(CellComparator.COMPARATOR) {
|
||||
private static final long serialVersionUID = 1L;
|
||||
{
|
||||
byte[] family = Bytes.toBytes("d");
|
||||
|
@ -157,7 +158,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
|
|||
assertTrue(
|
||||
format("Scan produced surprising result. expected: <%s>, actual: %s",
|
||||
expected, actual),
|
||||
KeyValue.COMPARATOR.compare(expected, actual) == 0);
|
||||
CellComparator.COMPARATOR.compare(expected, actual) == 0);
|
||||
}
|
||||
}
|
||||
assertFalse("Did not consume all expected values.", expectedIt.hasNext());
|
||||
|
|
|
@ -25,10 +25,9 @@ import java.nio.ByteBuffer;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
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.CellComparator.MetaCellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher;
|
||||
|
@ -58,7 +57,7 @@ import org.apache.hadoop.io.WritableUtils;
|
|||
* created and recycled by static PtEncoderFactory and PtDecoderFactory.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class PrefixTreeCodec implements DataBlockEncoder{
|
||||
public class PrefixTreeCodec implements DataBlockEncoder {
|
||||
|
||||
/**
|
||||
* no-arg constructor for reflection
|
||||
|
@ -150,12 +149,11 @@ public class PrefixTreeCodec implements DataBlockEncoder{
|
|||
* the way to this point.
|
||||
*/
|
||||
@Override
|
||||
public EncodedSeeker createSeeker(KVComparator comparator, HFileBlockDecodingContext decodingCtx) {
|
||||
if (comparator instanceof RawBytesComparator){
|
||||
throw new IllegalArgumentException("comparator must be KeyValue.KeyComparator");
|
||||
} else if (comparator instanceof MetaComparator){
|
||||
throw new IllegalArgumentException("DataBlockEncoding.PREFIX_TREE not compatible with hbase:meta "
|
||||
+"table");
|
||||
public EncodedSeeker createSeeker(CellComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx) {
|
||||
if (comparator instanceof MetaCellComparator) {
|
||||
throw new IllegalArgumentException(
|
||||
"DataBlockEncoding.PREFIX_TREE not compatible with hbase:meta " + "table");
|
||||
}
|
||||
|
||||
return new PrefixTreeSeeker(decodingCtx.getHFileContext().isIncludesMvcc());
|
||||
|
|
|
@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.codec.prefixtree;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
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;
|
||||
import org.apache.hadoop.hbase.SettableSequenceId;
|
||||
|
@ -212,13 +212,6 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
throw new RuntimeException("unexpected CellScannerPosition:" + position);
|
||||
}
|
||||
|
||||
@Override
|
||||
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());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int seekToKeyInBlock(Cell key, boolean forceBeforeOnExactMatch) {
|
||||
if (USE_POSITION_BEFORE) {
|
||||
|
@ -229,7 +222,8 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareKey(KVComparator comparator, Cell key) {
|
||||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
// can't optimize this, make a copy of the key
|
||||
ByteBuffer bb = getKeyDeepCopy();
|
||||
return comparator.compare(key,
|
||||
new KeyValue.KeyOnlyKeyValue(bb.array(), bb.arrayOffset(), bb.limit()));
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.codec.prefixtree.decode;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.column.ColumnReader;
|
||||
|
@ -61,7 +60,6 @@ public class PrefixTreeArrayScanner extends PrefixTreeCell implements CellScanne
|
|||
protected boolean nubCellsRemain;
|
||||
protected int currentCellIndex;
|
||||
|
||||
|
||||
/*********************** construct ******************************/
|
||||
|
||||
// pass in blockMeta so we can initialize buffers big enough for all cells in the block
|
||||
|
@ -420,7 +418,7 @@ public class PrefixTreeArrayScanner extends PrefixTreeCell implements CellScanne
|
|||
|
||||
protected int populateNonRowFieldsAndCompareTo(int cellNum, Cell key) {
|
||||
populateNonRowFields(cellNum);
|
||||
return CellComparator.compare(this, key, true);
|
||||
return comparator.compareKeyIgnoresMvcc(this, key);
|
||||
}
|
||||
|
||||
protected void populateFirstNonRowFields() {
|
||||
|
|
|
@ -25,6 +25,7 @@ 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.Bytes;
|
||||
|
||||
/**
|
||||
* As the PrefixTreeArrayScanner moves through the tree bytes, it changes the values in the fields
|
||||
|
@ -33,6 +34,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class PrefixTreeCell implements Cell, SettableSequenceId, Comparable<Cell> {
|
||||
// Create a reference here? Can be removed too
|
||||
protected CellComparator comparator = CellComparator.COMPARATOR;
|
||||
|
||||
/********************** static **********************/
|
||||
|
||||
|
@ -91,19 +94,35 @@ public class PrefixTreeCell implements Cell, SettableSequenceId, Comparable<Cell
|
|||
return false;
|
||||
}
|
||||
//Temporary hack to maintain backwards compatibility with KeyValue.equals
|
||||
return CellComparator.equalsIgnoreMvccVersion(this, (Cell)obj);
|
||||
return CellUtil.equalsIgnoreMvccVersion(this, (Cell)obj);
|
||||
|
||||
//TODO return CellComparator.equals(this, (Cell)obj);//see HBASE-6907
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return CellComparator.hashCodeIgnoreMvcc(this);
|
||||
return calculateHashForKey(this);
|
||||
}
|
||||
|
||||
private int calculateHashForKey(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 hash = 31 * rowHash + familyHash;
|
||||
hash = 31 * hash + qualifierHash;
|
||||
hash = 31 * hash + (int) cell.getTimestamp();
|
||||
hash = 31 * hash + cell.getTypeByte();
|
||||
return hash;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(Cell other) {
|
||||
return CellComparator.compare(this, other, false);
|
||||
return comparator.compare(this, other);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.row;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
|
||||
|
@ -37,7 +37,7 @@ public abstract class BaseTestRowData implements TestRowData {
|
|||
for (int i = 1; i < inputs.size(); ++i) {
|
||||
KeyValue lastKv = inputs.get(i - 1);
|
||||
KeyValue kv = inputs.get(i);
|
||||
if (!CellComparator.equalsRow(lastKv, kv)) {
|
||||
if (!CellUtil.matchingRow(lastKv, kv)) {
|
||||
rowStartIndexes.add(i);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
|
@ -85,7 +85,7 @@ public class TestPrefixTreeSearcher {
|
|||
// check all 3 permutations of equals()
|
||||
Assert.assertEquals(inputCell, outputCell);
|
||||
Assert.assertEquals(outputCell, inputCell);
|
||||
Assert.assertTrue(CellComparator.equals(inputCell, outputCell));
|
||||
Assert.assertTrue(CellUtil.equals(inputCell, outputCell));
|
||||
}
|
||||
Assert.assertEquals(rows.getInputs().size(), i + 1);
|
||||
} finally {
|
||||
|
@ -124,7 +124,7 @@ public class TestPrefixTreeSearcher {
|
|||
boolean hit = searcher.positionAt(kv);
|
||||
Assert.assertTrue(hit);
|
||||
Cell foundKv = searcher.current();
|
||||
Assert.assertTrue(CellComparator.equals(kv, foundKv));
|
||||
Assert.assertTrue(CellUtil.equals(kv, foundKv));
|
||||
}
|
||||
} finally {
|
||||
DecoderFactory.checkIn(searcher);
|
||||
|
|
|
@ -50,7 +50,7 @@ public class TestRowDataNumberStrings extends BaseTestRowData{
|
|||
|
||||
d.add(new KeyValue(row, family, column, 0L, Type.Put, value));
|
||||
}
|
||||
Collections.sort(d, new CellComparator());
|
||||
Collections.sort(d, CellComparator.COMPARATOR);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
|
||||
|
@ -73,11 +73,11 @@ public class TestRowDataSearcherRowMiss extends BaseTestRowData{
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
Cell first = searcher.current();
|
||||
Assert.assertTrue(CellComparator.equals(d.get(0), first));
|
||||
Assert.assertTrue(CellUtil.equals(d.get(0), first));
|
||||
|
||||
//test first cell in second row
|
||||
Assert.assertTrue(searcher.positionAt(d.get(1)));
|
||||
Assert.assertTrue(CellComparator.equals(d.get(1), searcher.current()));
|
||||
Assert.assertTrue(CellUtil.equals(d.get(1), searcher.current()));
|
||||
|
||||
testBetween1and2(searcher);
|
||||
testBetween2and3(searcher);
|
||||
|
@ -99,12 +99,12 @@ public class TestRowDataSearcherRowMiss extends BaseTestRowData{
|
|||
//test atOrBefore
|
||||
p = searcher.positionAtOrBefore(betweenAAndAAA);
|
||||
Assert.assertEquals(CellScannerPosition.BEFORE, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(1)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(1)));
|
||||
|
||||
//test atOrAfter
|
||||
p = searcher.positionAtOrAfter(betweenAAndAAA);
|
||||
Assert.assertEquals(CellScannerPosition.AFTER, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(2)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(2)));
|
||||
}
|
||||
|
||||
private void testBetween2and3(CellSearcher searcher){
|
||||
|
@ -117,12 +117,12 @@ public class TestRowDataSearcherRowMiss extends BaseTestRowData{
|
|||
//test atOrBefore
|
||||
p = searcher.positionAtOrBefore(betweenAAAndB);
|
||||
Assert.assertEquals(CellScannerPosition.BEFORE, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(2)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(2)));
|
||||
|
||||
//test atOrAfter
|
||||
p = searcher.positionAtOrAfter(betweenAAAndB);
|
||||
Assert.assertEquals(CellScannerPosition.AFTER, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(3)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(3)));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
|
||||
|
@ -73,11 +73,11 @@ public class TestRowDataSimple extends BaseTestRowData {
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
Cell first = searcher.current();
|
||||
Assert.assertTrue(CellComparator.equals(d.get(0), first));
|
||||
Assert.assertTrue(CellUtil.equals(d.get(0), first));
|
||||
|
||||
// test first cell in second row
|
||||
Assert.assertTrue(searcher.positionAt(d.get(3)));
|
||||
Assert.assertTrue(CellComparator.equals(d.get(3), searcher.current()));
|
||||
Assert.assertTrue(CellUtil.equals(d.get(3), searcher.current()));
|
||||
|
||||
Cell between4And5 = new KeyValue(rowB, cf, cq1, ts - 2, v0);
|
||||
|
||||
|
@ -87,12 +87,12 @@ public class TestRowDataSimple extends BaseTestRowData {
|
|||
// test atOrBefore
|
||||
p = searcher.positionAtOrBefore(between4And5);
|
||||
Assert.assertEquals(CellScannerPosition.BEFORE, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(4)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(4)));
|
||||
|
||||
// test atOrAfter
|
||||
p = searcher.positionAtOrAfter(between4And5);
|
||||
Assert.assertEquals(CellScannerPosition.AFTER, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(5)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(5)));
|
||||
|
||||
// test when key falls before first key in block
|
||||
Cell beforeFirst = new KeyValue(Bytes.toBytes("A"), cf, cq0, ts, v0);
|
||||
|
@ -101,7 +101,7 @@ public class TestRowDataSimple extends BaseTestRowData {
|
|||
Assert.assertEquals(CellScannerPosition.BEFORE_FIRST, p);
|
||||
p = searcher.positionAtOrAfter(beforeFirst);
|
||||
Assert.assertEquals(CellScannerPosition.AFTER, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), d.get(0)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), d.get(0)));
|
||||
Assert.assertEquals(d.get(0), searcher.current());
|
||||
|
||||
// test when key falls after last key in block
|
||||
|
@ -111,7 +111,7 @@ public class TestRowDataSimple extends BaseTestRowData {
|
|||
Assert.assertEquals(CellScannerPosition.AFTER_LAST, p);
|
||||
p = searcher.positionAtOrBefore(afterLast);
|
||||
Assert.assertEquals(CellScannerPosition.BEFORE, p);
|
||||
Assert.assertTrue(CellComparator.equals(searcher.current(), CollectionUtils.getLast(d)));
|
||||
Assert.assertTrue(CellUtil.equals(searcher.current(), CollectionUtils.getLast(d)));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -164,8 +164,8 @@ 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().compare(splitCell, bb.array(), bb.arrayOffset(),
|
||||
bb.limit()) <= 0) {
|
||||
atEnd = true;
|
||||
return false;
|
||||
}
|
||||
|
@ -176,7 +176,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
|
|||
@Override
|
||||
public boolean seekTo() throws IOException {
|
||||
if (top) {
|
||||
int r = this.delegate.seekTo(new KeyValue.KeyOnlyKeyValue(splitkey, 0, splitkey.length));
|
||||
int r = this.delegate.seekTo(splitCell);
|
||||
if (r == HConstants.INDEX_KEY_MAGIC) {
|
||||
return true;
|
||||
}
|
||||
|
@ -196,9 +196,9 @@ 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().
|
||||
compare(splitCell, k.array(), k.arrayOffset(), k.limit()
|
||||
)) > 0;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
|
||||
|
@ -212,11 +212,11 @@ public class HalfStoreFileReader extends StoreFile.Reader {
|
|||
@Override
|
||||
public int seekTo(Cell key) throws IOException {
|
||||
if (top) {
|
||||
if (getComparator().compareOnlyKeyPortion(key, splitCell) < 0) {
|
||||
if (getComparator().compareKeyIgnoresMvcc(key, splitCell) < 0) {
|
||||
return -1;
|
||||
}
|
||||
} else {
|
||||
if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
|
||||
if (getComparator().compareKeyIgnoresMvcc(key, splitCell) >= 0) {
|
||||
// we would place the scanner in the second half.
|
||||
// it might be an error to return false here ever...
|
||||
boolean res = delegate.seekBefore(splitCell);
|
||||
|
@ -237,11 +237,11 @@ public class HalfStoreFileReader extends StoreFile.Reader {
|
|||
// except
|
||||
// that we call reseekTo (and not seekTo) on the delegate.
|
||||
if (top) {
|
||||
if (getComparator().compareOnlyKeyPortion(key, splitCell) < 0) {
|
||||
if (getComparator().compareKeyIgnoresMvcc(key, splitCell) < 0) {
|
||||
return -1;
|
||||
}
|
||||
} else {
|
||||
if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
|
||||
if (getComparator().compareKeyIgnoresMvcc(key, splitCell) >= 0) {
|
||||
// we would place the scanner in the second half.
|
||||
// it might be an error to return false here ever...
|
||||
boolean res = delegate.seekBefore(splitCell);
|
||||
|
@ -263,13 +263,13 @@ public class HalfStoreFileReader extends StoreFile.Reader {
|
|||
public boolean seekBefore(Cell key) throws IOException {
|
||||
if (top) {
|
||||
Cell fk = new KeyValue.KeyOnlyKeyValue(getFirstKey(), 0, getFirstKey().length);
|
||||
if (getComparator().compareOnlyKeyPortion(key, fk) <= 0) {
|
||||
if (getComparator().compareKeyIgnoresMvcc(key, fk) <= 0) {
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
// The equals sign isn't strictly necessary just here to be consistent
|
||||
// with seekTo
|
||||
if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
|
||||
if (getComparator().compareKeyIgnoresMvcc(key, splitCell) >= 0) {
|
||||
boolean ret = this.delegate.seekBefore(splitCell);
|
||||
if (ret) {
|
||||
atEnd = false;
|
||||
|
|
|
@ -27,10 +27,11 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.CellComparator.MetaCellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -109,7 +110,7 @@ public class FixedFileTrailer {
|
|||
|
||||
/** Raw key comparator class name in version 3 */
|
||||
// We could write the actual class name from 2.0 onwards and handle BC
|
||||
private String comparatorClassName = KeyValue.COMPARATOR.getClass().getName();
|
||||
private String comparatorClassName = CellComparator.COMPARATOR.getClass().getName();
|
||||
|
||||
/** The encryption key */
|
||||
private byte[] encryptionKey;
|
||||
|
@ -538,12 +539,12 @@ public class FixedFileTrailer {
|
|||
return minorVersion;
|
||||
}
|
||||
|
||||
public void setComparatorClass(Class<? extends KVComparator> klass) {
|
||||
public void setComparatorClass(Class<? extends CellComparator> klass) {
|
||||
// Is the comparator instantiable?
|
||||
try {
|
||||
// If null, it should be the Bytes.BYTES_RAWCOMPARATOR
|
||||
if (klass != null) {
|
||||
KVComparator comp = klass.newInstance();
|
||||
CellComparator comp = klass.newInstance();
|
||||
// if the name wasn't one of the legacy names, maybe its a legit new
|
||||
// kind of comparator.
|
||||
comparatorClassName = klass.getName();
|
||||
|
@ -555,34 +556,35 @@ public class FixedFileTrailer {
|
|||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static Class<? extends KVComparator> getComparatorClass(
|
||||
String comparatorClassName) throws IOException {
|
||||
try {
|
||||
// HFile V2 legacy comparator class names.
|
||||
if (comparatorClassName.equals(KeyValue.COMPARATOR.getLegacyKeyComparatorName())) {
|
||||
comparatorClassName = KeyValue.COMPARATOR.getClass().getName();
|
||||
} else if (comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())) {
|
||||
comparatorClassName = KeyValue.META_COMPARATOR.getClass().getName();
|
||||
} else if (comparatorClassName.equals(KeyValue.RAW_COMPARATOR.getLegacyKeyComparatorName())) {
|
||||
return null;
|
||||
}
|
||||
|
||||
private static Class<? extends CellComparator> getComparatorClass(String comparatorClassName)
|
||||
throws IOException {
|
||||
Class<? extends CellComparator> comparatorKlass;
|
||||
if (comparatorClassName.equals(KeyValue.COMPARATOR.getLegacyKeyComparatorName())
|
||||
|| comparatorClassName.equals(KeyValue.COMPARATOR.getClass().getName())) {
|
||||
comparatorKlass = CellComparator.class;
|
||||
} else if (comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())
|
||||
|| comparatorClassName.equals(KeyValue.META_COMPARATOR.getClass().getName())) {
|
||||
comparatorKlass = MetaCellComparator.class;
|
||||
} else if (comparatorClassName.equals(KeyValue.RAW_COMPARATOR.getClass().getName())
|
||||
|| comparatorClassName.equals(KeyValue.RAW_COMPARATOR.getLegacyKeyComparatorName())) {
|
||||
// When the comparator to be used is Bytes.BYTES_RAWCOMPARATOR, we just return null from here
|
||||
// Bytes.BYTES_RAWCOMPARATOR is not a CellComparator
|
||||
comparatorKlass = null;
|
||||
} else {
|
||||
// if the name wasn't one of the legacy names, maybe its a legit new kind of comparator.
|
||||
if (comparatorClassName.equals(KeyValue.RAW_COMPARATOR.getClass().getName())) {
|
||||
// Return null for Bytes.BYTES_RAWCOMPARATOR
|
||||
return null;
|
||||
} else {
|
||||
return (Class<? extends KVComparator>) Class.forName(comparatorClassName);
|
||||
try {
|
||||
comparatorKlass = (Class<? extends CellComparator>) Class.forName(comparatorClassName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
} catch (ClassNotFoundException ex) {
|
||||
throw new IOException(ex);
|
||||
}
|
||||
return comparatorKlass;
|
||||
}
|
||||
|
||||
public static KVComparator createComparator(
|
||||
public static CellComparator createComparator(
|
||||
String comparatorClassName) throws IOException {
|
||||
try {
|
||||
Class<? extends KVComparator> comparatorClass = getComparatorClass(comparatorClassName);
|
||||
Class<? extends CellComparator> comparatorClass = getComparatorClass(comparatorClassName);
|
||||
return comparatorClass != null ? comparatorClass.newInstance() : null;
|
||||
} catch (InstantiationException e) {
|
||||
throw new IOException("Comparator class " + comparatorClassName +
|
||||
|
@ -593,7 +595,7 @@ public class FixedFileTrailer {
|
|||
}
|
||||
}
|
||||
|
||||
KVComparator createComparator() throws IOException {
|
||||
CellComparator createComparator() throws IOException {
|
||||
expectAtLeastMajorVersion(2);
|
||||
return createComparator(comparatorClassName);
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import java.util.SortedMap;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -48,9 +49,8 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
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.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
|
@ -61,7 +61,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterWriter;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ChecksumType;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -252,7 +251,8 @@ public class HFile {
|
|||
protected FileSystem fs;
|
||||
protected Path path;
|
||||
protected FSDataOutputStream ostream;
|
||||
protected KVComparator comparator = KeyValue.COMPARATOR;
|
||||
protected CellComparator comparator =
|
||||
CellComparator.COMPARATOR;
|
||||
protected InetSocketAddress[] favoredNodes;
|
||||
private HFileContext fileContext;
|
||||
|
||||
|
@ -275,7 +275,7 @@ public class HFile {
|
|||
return this;
|
||||
}
|
||||
|
||||
public WriterFactory withComparator(KVComparator comparator) {
|
||||
public WriterFactory withComparator(CellComparator comparator) {
|
||||
Preconditions.checkNotNull(comparator);
|
||||
this.comparator = comparator;
|
||||
return this;
|
||||
|
@ -305,7 +305,7 @@ public class HFile {
|
|||
}
|
||||
|
||||
protected abstract Writer createWriter(FileSystem fs, Path path, FSDataOutputStream ostream,
|
||||
KVComparator comparator, HFileContext fileContext) throws IOException;
|
||||
CellComparator comparator, HFileContext fileContext) throws IOException;
|
||||
}
|
||||
|
||||
/** The configuration key for HFile version to use for new files */
|
||||
|
@ -388,7 +388,7 @@ public class HFile {
|
|||
*/
|
||||
String getName();
|
||||
|
||||
KVComparator getComparator();
|
||||
CellComparator getComparator();
|
||||
|
||||
HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction);
|
||||
|
||||
|
|
|
@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
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.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
|
@ -107,9 +107,10 @@ public class HFileBlockIndex {
|
|||
*/
|
||||
public static class BlockIndexReader implements HeapSize {
|
||||
/** Needed doing lookup on blocks. */
|
||||
private final KVComparator comparator;
|
||||
private final CellComparator comparator;
|
||||
|
||||
// Root-level data.
|
||||
// TODO : Convert these to Cells (ie) KeyValue.KeyOnlyKV
|
||||
private byte[][] blockKeys;
|
||||
private long[] blockOffsets;
|
||||
private int[] blockDataSizes;
|
||||
|
@ -132,14 +133,15 @@ public class HFileBlockIndex {
|
|||
/** A way to read {@link HFile} blocks at a given offset */
|
||||
private CachingBlockReader cachingBlockReader;
|
||||
|
||||
public BlockIndexReader(final KVComparator c, final int treeLevel,
|
||||
public BlockIndexReader(final CellComparator c, final int treeLevel,
|
||||
final CachingBlockReader cachingBlockReader) {
|
||||
this(c, treeLevel);
|
||||
this.cachingBlockReader = cachingBlockReader;
|
||||
}
|
||||
|
||||
public BlockIndexReader(final KVComparator c, final int treeLevel)
|
||||
public BlockIndexReader(final CellComparator c, final int treeLevel)
|
||||
{
|
||||
// Can be null for METAINDEX block
|
||||
comparator = c;
|
||||
searchTreeLevel = treeLevel;
|
||||
}
|
||||
|
@ -388,15 +390,21 @@ public class HFileBlockIndex {
|
|||
|
||||
/**
|
||||
* Finds the root-level index block containing the given key.
|
||||
*
|
||||
*
|
||||
* @param key
|
||||
* Key to find
|
||||
* @param comp
|
||||
* the comparator to be used
|
||||
* @return Offset of block containing <code>key</code> (between 0 and the
|
||||
* number of blocks - 1) or -1 if this file does not contain the
|
||||
* request.
|
||||
*/
|
||||
public int rootBlockContainingKey(final byte[] key, int offset, int length) {
|
||||
int pos = Bytes.binarySearch(blockKeys, key, offset, length, comparator);
|
||||
// When we want to find the meta index block or bloom block for ROW bloom
|
||||
// type Bytes.BYTES_RAWCOMPARATOR would be enough. For the ROW_COL bloom case we need the
|
||||
// CellComparator.
|
||||
public int rootBlockContainingKey(final byte[] key, int offset, int length,
|
||||
CellComparator comp) {
|
||||
int pos = Bytes.binarySearch(blockKeys, key, offset, length, comp);
|
||||
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
|
||||
// binarySearch's javadoc.
|
||||
|
||||
|
@ -416,6 +424,23 @@ public class HFileBlockIndex {
|
|||
return i - 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the root-level index block containing the given key.
|
||||
*
|
||||
* @param key
|
||||
* Key to find
|
||||
* @return Offset of block containing <code>key</code> (between 0 and the
|
||||
* number of blocks - 1) or -1 if this file does not contain the
|
||||
* request.
|
||||
*/
|
||||
// When we want to find the meta index block or bloom block for ROW bloom
|
||||
// type
|
||||
// Bytes.BYTES_RAWCOMPARATOR would be enough. For the ROW_COL bloom case we
|
||||
// need the CellComparator.
|
||||
public int rootBlockContainingKey(final byte[] key, int offset, int length) {
|
||||
return rootBlockContainingKey(key, offset, length, comparator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the root-level index block containing the given key.
|
||||
*
|
||||
|
@ -423,6 +448,7 @@ public class HFileBlockIndex {
|
|||
* Key to find
|
||||
*/
|
||||
public int rootBlockContainingKey(final Cell key) {
|
||||
// Here the comparator should not be null as this happens for the root-level block
|
||||
int pos = Bytes.binarySearch(blockKeys, key, comparator);
|
||||
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
|
||||
// binarySearch's javadoc.
|
||||
|
@ -487,6 +513,7 @@ public class HFileBlockIndex {
|
|||
int targetKeyLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (i + 2)) -
|
||||
targetKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD;
|
||||
|
||||
// TODO check whether we can make BB backed Cell here? So can avoid bytes copy.
|
||||
return ByteBufferUtils.toBytes(nonRootIndex, targetKeyOffset, targetKeyLength);
|
||||
}
|
||||
|
||||
|
@ -507,7 +534,7 @@ public class HFileBlockIndex {
|
|||
* @throws IOException
|
||||
*/
|
||||
static int binarySearchNonRootIndex(Cell key, ByteBuffer nonRootIndex,
|
||||
KVComparator comparator) {
|
||||
CellComparator comparator) {
|
||||
|
||||
int numEntries = nonRootIndex.getInt(0);
|
||||
int low = 0;
|
||||
|
@ -544,9 +571,10 @@ public class HFileBlockIndex {
|
|||
// 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
|
||||
// TODO avaoid array call.
|
||||
nonRootIndexKV.setKey(nonRootIndex.array(),
|
||||
nonRootIndex.arrayOffset() + midKeyOffset, midLength);
|
||||
int cmp = comparator.compareOnlyKeyPortion(key, nonRootIndexKV);
|
||||
int cmp = comparator.compareKeyIgnoresMvcc(key, nonRootIndexKV);
|
||||
|
||||
// key lives above the midpoint
|
||||
if (cmp > 0)
|
||||
|
@ -596,7 +624,7 @@ public class HFileBlockIndex {
|
|||
*
|
||||
*/
|
||||
static int locateNonRootIndexEntry(ByteBuffer nonRootBlock, Cell key,
|
||||
KVComparator comparator) {
|
||||
CellComparator comparator) {
|
||||
int entryIndex = binarySearchNonRootIndex(key, nonRootBlock, comparator);
|
||||
|
||||
if (entryIndex != -1) {
|
||||
|
@ -641,7 +669,7 @@ public class HFileBlockIndex {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Read in the root-level index from the given input stream. Must match
|
||||
* what was written into the root level by
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
|
@ -317,8 +316,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
do {
|
||||
Cell cell = scanner.getKeyValue();
|
||||
if (row != null && row.length != 0) {
|
||||
int result = CellComparator.compareRows(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), row, 0, row.length);
|
||||
int result = CellComparator.COMPARATOR.compareRows(cell, row, 0, row.length);
|
||||
if (result > 0) {
|
||||
break;
|
||||
} else if (result < 0) {
|
||||
|
@ -348,7 +346,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
}
|
||||
// check if rows are in order
|
||||
if (checkRow && pCell != null) {
|
||||
if (CellComparator.compareRows(pCell, cell) > 0) {
|
||||
if (CellComparator.COMPARATOR.compareRows(pCell, cell) > 0) {
|
||||
System.err.println("WARNING, previous row is greater then"
|
||||
+ " current row\n\tfilename -> " + file + "\n\tprevious -> "
|
||||
+ CellUtil.getCellKeyAsString(pCell) + "\n\tcurrent -> "
|
||||
|
@ -468,7 +466,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
public void collect(Cell cell) {
|
||||
valLen.update(cell.getValueLength());
|
||||
if (prevCell != null &&
|
||||
KeyValue.COMPARATOR.compareRows(prevCell, cell) != 0) {
|
||||
CellComparator.COMPARATOR.compareRows(prevCell, cell) != 0) {
|
||||
// new row
|
||||
collectRow();
|
||||
}
|
||||
|
|
|
@ -33,9 +33,11 @@ import org.apache.hadoop.conf.Configurable;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
|
@ -97,7 +99,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
private int avgValueLen = -1;
|
||||
|
||||
/** Key comparator */
|
||||
private KVComparator comparator = new KVComparator();
|
||||
private CellComparator comparator = CellComparator.COMPARATOR;
|
||||
|
||||
/** Size of this file. */
|
||||
private final long fileSize;
|
||||
|
@ -190,7 +192,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
dataBlockIndexReader = new HFileBlockIndex.BlockIndexReader(comparator,
|
||||
trailer.getNumDataIndexLevels(), this);
|
||||
metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader(
|
||||
KeyValue.RAW_COMPARATOR, 1);
|
||||
null, 1);
|
||||
|
||||
// Parse load-on-open data.
|
||||
|
||||
|
@ -356,7 +358,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
@Override
|
||||
public byte[] getFirstRowKey() {
|
||||
byte[] firstKey = getFirstKey();
|
||||
return firstKey == null? null: KeyValue.createKeyValueFromKey(firstKey).getRow();
|
||||
return firstKey == null? null: KeyValueUtil.createKeyValueFromKey(firstKey).getRow();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -368,7 +370,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
@Override
|
||||
public byte[] getLastRowKey() {
|
||||
byte[] lastKey = getLastKey();
|
||||
return lastKey == null? null: KeyValue.createKeyValueFromKey(lastKey).getRow();
|
||||
return lastKey == null? null: KeyValueUtil.createKeyValueFromKey(lastKey).getRow();
|
||||
}
|
||||
|
||||
/** @return number of KV entries in this HFile */
|
||||
|
@ -379,7 +381,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
|
||||
/** @return comparator */
|
||||
@Override
|
||||
public KVComparator getComparator() {
|
||||
public CellComparator getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
|
@ -633,7 +635,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
int keyOffset =
|
||||
blockBuffer.arrayOffset() + blockBuffer.position() + (Bytes.SIZEOF_INT * 2);
|
||||
keyOnlyKv.setKey(blockBuffer.array(), keyOffset, klen);
|
||||
int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlyKv);
|
||||
int comp = reader.getComparator().compareKeyIgnoresMvcc(key, keyOnlyKv);
|
||||
|
||||
if (comp == 0) {
|
||||
if (seekBefore) {
|
||||
|
@ -706,7 +708,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
// 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)) {
|
||||
.getComparator().compareKeyIgnoresMvcc(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
|
||||
|
@ -758,7 +760,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock);
|
||||
|
||||
if (reader.getComparator()
|
||||
.compareOnlyKeyPortion(
|
||||
.compareKeyIgnoresMvcc(
|
||||
new KeyValue.KeyOnlyKeyValue(firstKey.array(), firstKey.arrayOffset(),
|
||||
firstKey.limit()), key) >= 0) {
|
||||
long previousBlockOffset = seekToBlock.getPrevBlockOffset();
|
||||
|
@ -850,9 +852,14 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
+ KEY_VALUE_LEN_SIZE, currKeyLen).slice();
|
||||
}
|
||||
|
||||
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);
|
||||
public int compareKey(CellComparator comparator, byte[] key, int offset, int length) {
|
||||
// TODO HFileScannerImpl, instance will be used by single thread alone. So we can
|
||||
// have one KeyValue.KeyOnlyKeyValue instance as instance variable and reuse here and in
|
||||
// compareKey(CellComparator comparator, Cell key), seekBefore(Cell key) and
|
||||
// blockSeek(Cell key, boolean seekBefore)
|
||||
KeyValue.KeyOnlyKeyValue keyOnlyKv = new KeyValue.KeyOnlyKeyValue(key, offset, length);
|
||||
return comparator.compare(keyOnlyKv, blockBuffer.array(), blockBuffer.arrayOffset()
|
||||
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1064,8 +1071,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
currValueLen);
|
||||
}
|
||||
|
||||
public int compareKey(KVComparator comparator, Cell key) {
|
||||
return comparator.compareOnlyKeyPortion(
|
||||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
return comparator.compareKeyIgnoresMvcc(
|
||||
key,
|
||||
new KeyValue.KeyOnlyKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
|
||||
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen));
|
||||
|
@ -1516,10 +1523,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
return seeker.getKeyDeepCopy();
|
||||
}
|
||||
|
||||
public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
return seeker.compareKey(comparator, key, offset, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer getValue() {
|
||||
assertValidSeek();
|
||||
|
@ -1569,7 +1572,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
return seeker.seekToKeyInBlock(key, seekBefore);
|
||||
}
|
||||
|
||||
public int compareKey(KVComparator comparator, Cell key) {
|
||||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
return seeker.compareKey(comparator, key);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
|
||||
public class HFileWriterFactory extends HFile.WriterFactory {
|
||||
HFileWriterFactory(Configuration conf, CacheConfig cacheConf) {
|
||||
|
@ -33,7 +33,7 @@ public class HFileWriterFactory extends HFile.WriterFactory {
|
|||
|
||||
@Override
|
||||
public HFile.Writer createWriter(FileSystem fs, Path path, FSDataOutputStream ostream,
|
||||
KVComparator comparator, HFileContext context)
|
||||
CellComparator comparator, HFileContext context)
|
||||
throws IOException {
|
||||
return new HFileWriterImpl(conf, cacheConf, path, ostream, comparator, context);
|
||||
}
|
||||
|
|
|
@ -36,8 +36,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator.MetaCellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
|
@ -83,7 +82,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
protected long totalUncompressedBytes = 0;
|
||||
|
||||
/** Key comparator. Used to ensure we write in order. */
|
||||
protected final KVComparator comparator;
|
||||
protected final CellComparator comparator;
|
||||
|
||||
/** Meta block names. */
|
||||
protected List<byte[]> metaNames = new ArrayList<byte[]>();
|
||||
|
@ -154,7 +153,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
|
||||
public HFileWriterImpl(final Configuration conf, CacheConfig cacheConf, Path path,
|
||||
FSDataOutputStream outputStream,
|
||||
KVComparator comparator, HFileContext fileContext) {
|
||||
CellComparator comparator, HFileContext fileContext) {
|
||||
this.outputStream = outputStream;
|
||||
this.path = path;
|
||||
this.name = path != null ? path.getName() : outputStream.toString();
|
||||
|
@ -166,7 +165,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
this.blockEncoder = NoOpDataBlockEncoder.INSTANCE;
|
||||
}
|
||||
this.comparator = comparator != null ? comparator
|
||||
: KeyValue.COMPARATOR;
|
||||
: CellComparator.COMPARATOR;
|
||||
|
||||
closeOutputStream = path != null;
|
||||
this.cacheConf = cacheConf;
|
||||
|
@ -224,7 +223,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
throw new IOException("Key cannot be null or empty");
|
||||
}
|
||||
if (lastCell != null) {
|
||||
int keyComp = comparator.compareOnlyKeyPortion(lastCell, cell);
|
||||
int keyComp = comparator.compareKeyIgnoresMvcc(lastCell, cell);
|
||||
|
||||
if (keyComp > 0) {
|
||||
throw new IOException("Added a key not lexically larger than"
|
||||
|
@ -319,7 +318,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
fsBlockWriter.writeHeaderAndData(outputStream);
|
||||
int onDiskSize = fsBlockWriter.getOnDiskSizeWithHeader();
|
||||
Cell indexEntry =
|
||||
CellComparator.getMidpoint(this.comparator, lastCellOfPreviousBlock, firstCellInBlock);
|
||||
getMidpoint(this.comparator, lastCellOfPreviousBlock, firstCellInBlock);
|
||||
dataBlockIndexWriter.addEntry(CellUtil.getCellKeySerializedAsKeyValueKey(indexEntry),
|
||||
lastDataBlockOffset, onDiskSize);
|
||||
totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
|
||||
|
@ -327,6 +326,134 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
doCacheOnWrite(lastDataBlockOffset);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Try to return a Cell that falls between <code>left</code> and
|
||||
* <code>right</code> but that is shorter; i.e. takes up less space. This
|
||||
* trick is used building HFile block index. Its an optimization. It does not
|
||||
* always work. In this case we'll just return the <code>right</code> cell.
|
||||
*
|
||||
* @param comparator
|
||||
* Comparator to use.
|
||||
* @param left
|
||||
* @param right
|
||||
* @return A cell that sorts between <code>left</code> and <code>right</code>.
|
||||
*/
|
||||
public static Cell getMidpoint(final CellComparator comparator, final Cell left,
|
||||
final Cell right) {
|
||||
// TODO: Redo so only a single pass over the arrays rather than one to
|
||||
// compare and then a
|
||||
// second composing midpoint.
|
||||
if (right == null) {
|
||||
throw new IllegalArgumentException("right cell can not be null");
|
||||
}
|
||||
if (left == null) {
|
||||
return right;
|
||||
}
|
||||
// If Cells from meta table, don't mess around. meta table Cells have schema
|
||||
// (table,startrow,hash) so can't be treated as plain byte arrays. Just skip
|
||||
// out without
|
||||
// trying to do this optimization.
|
||||
if (comparator != null && comparator instanceof MetaCellComparator) {
|
||||
return right;
|
||||
}
|
||||
int diff = comparator.compareRows(left, right);
|
||||
if (diff > 0) {
|
||||
throw new IllegalArgumentException("Left row sorts after right row; left="
|
||||
+ CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
|
||||
}
|
||||
if (diff < 0) {
|
||||
// Left row is < right row.
|
||||
byte[] midRow = getMinimumMidpointArray(left.getRowArray(), left.getRowOffset(),
|
||||
left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
// If midRow is null, just return 'right'. Can't do optimization.
|
||||
if (midRow == null)
|
||||
return right;
|
||||
return CellUtil.createCell(midRow);
|
||||
}
|
||||
// Rows are same. Compare on families.
|
||||
int lFamOffset = left.getFamilyOffset();
|
||||
int rFamOffset = right.getFamilyOffset();
|
||||
int lFamLength = left.getFamilyLength();
|
||||
int rFamLength = right.getFamilyLength();
|
||||
diff = CellComparator.compareFamilies(left, right);
|
||||
if (diff > 0) {
|
||||
throw new IllegalArgumentException("Left family sorts after right family; left="
|
||||
+ CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
|
||||
}
|
||||
if (diff < 0) {
|
||||
byte[] midRow = getMinimumMidpointArray(left.getFamilyArray(), lFamOffset,
|
||||
lFamLength, right.getFamilyArray(), rFamOffset,
|
||||
rFamLength);
|
||||
// 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 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);
|
||||
}
|
||||
// Families are same. Compare on qualifiers.
|
||||
diff = CellComparator.compareQualifiers(left, right);
|
||||
if (diff > 0) {
|
||||
throw new IllegalArgumentException("Left qualifier sorts after right qualifier; left="
|
||||
+ 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.
|
||||
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);
|
||||
}
|
||||
// No opportunity for optimization. Just return right key.
|
||||
return right;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param leftArray
|
||||
* @param leftOffset
|
||||
* @param leftLength
|
||||
* @param rightArray
|
||||
* @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) {
|
||||
// rows are different
|
||||
int minLength = leftLength < rightLength ? leftLength : rightLength;
|
||||
short diffIdx = 0;
|
||||
while (diffIdx < minLength
|
||||
&& leftArray[leftOffset + diffIdx] == rightArray[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);
|
||||
} else {
|
||||
int diffByte = leftArray[leftOffset + diffIdx];
|
||||
if ((0xff & diffByte) < 0xff && (diffByte + 1) < (rightArray[rightOffset + diffIdx] & 0xff)) {
|
||||
minimumMidpointArray = new byte[diffIdx + 1];
|
||||
System.arraycopy(leftArray, leftOffset, minimumMidpointArray, 0, diffIdx);
|
||||
minimumMidpointArray[diffIdx] = (byte) (diffByte + 1);
|
||||
} else {
|
||||
minimumMidpointArray = new byte[diffIdx + 1];
|
||||
System.arraycopy(rightArray, rightOffset, minimumMidpointArray, 0, diffIdx + 1);
|
||||
}
|
||||
}
|
||||
return minimumMidpointArray;
|
||||
}
|
||||
|
||||
/** Gives inline block writers an opportunity to contribute blocks. */
|
||||
private void writeInlineBlocks(boolean closing) throws IOException {
|
||||
|
|
|
@ -37,6 +37,7 @@ 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.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -248,7 +249,7 @@ public class HFileOutputFormat2
|
|||
|
||||
wl.writer = new StoreFile.WriterBuilder(conf, new CacheConfig(tempConf), fs)
|
||||
.withOutputDir(familydir).withBloomType(bloomType)
|
||||
.withComparator(KeyValue.COMPARATOR)
|
||||
.withComparator(CellComparator.COMPARATOR)
|
||||
.withFileContext(hFileContext).build();
|
||||
|
||||
this.writers.put(family, wl);
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.io.serializer.Deserializer;
|
||||
import org.apache.hadoop.io.serializer.Serialization;
|
||||
import org.apache.hadoop.io.serializer.Serializer;
|
||||
|
@ -55,7 +56,7 @@ public class KeyValueSerialization implements Serialization<KeyValue> {
|
|||
@Override
|
||||
public KeyValue deserialize(KeyValue ignore) throws IOException {
|
||||
// I can't overwrite the passed in KV, not from a proto kv, not just yet. TODO
|
||||
return KeyValue.create(this.dis);
|
||||
return KeyValueUtil.create(this.dis);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -79,7 +80,7 @@ public class KeyValueSerialization implements Serialization<KeyValue> {
|
|||
|
||||
@Override
|
||||
public void serialize(KeyValue kv) throws IOException {
|
||||
KeyValue.write(kv, this.dos);
|
||||
KeyValueUtil.write(kv, this.dos);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.mapreduce.Reducer;
|
||||
|
@ -39,7 +40,7 @@ public class KeyValueSortReducer extends Reducer<ImmutableBytesWritable, KeyValu
|
|||
protected void reduce(ImmutableBytesWritable row, java.lang.Iterable<KeyValue> kvs,
|
||||
org.apache.hadoop.mapreduce.Reducer<ImmutableBytesWritable, KeyValue, ImmutableBytesWritable, KeyValue>.Context context)
|
||||
throws java.io.IOException, InterruptedException {
|
||||
TreeSet<KeyValue> map = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
|
||||
TreeSet<KeyValue> map = new TreeSet<KeyValue>(CellComparator.COMPARATOR);
|
||||
for (KeyValue kv: kvs) {
|
||||
try {
|
||||
map.add(kv.clone());
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.TreeSet;
|
|||
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.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -58,7 +59,7 @@ public class PutSortReducer extends
|
|||
"putsortreducer.row.threshold", 1L * (1<<30));
|
||||
Iterator<Put> iter = puts.iterator();
|
||||
while (iter.hasNext()) {
|
||||
TreeSet<KeyValue> map = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
|
||||
TreeSet<KeyValue> map = new TreeSet<KeyValue>(CellComparator.COMPARATOR);
|
||||
long curSize = 0;
|
||||
// stop at the end or the RAM threshold
|
||||
while (iter.hasNext() && curSize < threshold) {
|
||||
|
|
|
@ -108,7 +108,7 @@ implements Configurable {
|
|||
}
|
||||
}
|
||||
int pos = Bytes.binarySearch(this.splits, key.get(), key.getOffset(),
|
||||
key.getLength(), Bytes.BYTES_RAWCOMPARATOR);
|
||||
key.getLength());
|
||||
// Below code is from hfile index search.
|
||||
if (pos < 0) {
|
||||
pos++;
|
||||
|
|
|
@ -28,6 +28,7 @@ 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.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
|
@ -145,7 +146,7 @@ public class TextSortReducer extends
|
|||
"reducer.row.threshold", 1L * (1<<30));
|
||||
Iterator<Text> iter = lines.iterator();
|
||||
while (iter.hasNext()) {
|
||||
Set<KeyValue> kvs = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
|
||||
Set<KeyValue> kvs = new TreeSet<KeyValue>(CellComparator.COMPARATOR);
|
||||
long curSize = 0;
|
||||
// stop at the end or the RAM threshold
|
||||
while (iter.hasNext() && curSize < threshold) {
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.concurrent.ConcurrentNavigableMap;
|
|||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
public class CellSkipListSet implements NavigableSet<Cell> {
|
||||
private final ConcurrentNavigableMap<Cell, Cell> delegatee;
|
||||
|
||||
CellSkipListSet(final KeyValue.KVComparator c) {
|
||||
CellSkipListSet(final CellComparator c) {
|
||||
this.delegatee = new ConcurrentSkipListMap<Cell, Cell>(c);
|
||||
}
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -84,7 +85,7 @@ public class DefaultMemStore implements MemStore {
|
|||
// Snapshot of memstore. Made for flusher.
|
||||
volatile CellSkipListSet snapshot;
|
||||
|
||||
final KeyValue.KVComparator comparator;
|
||||
final CellComparator comparator;
|
||||
|
||||
// Used to track own heapSize
|
||||
final AtomicLong size;
|
||||
|
@ -104,7 +105,7 @@ public class DefaultMemStore implements MemStore {
|
|||
* Default constructor. Used for tests.
|
||||
*/
|
||||
public DefaultMemStore() {
|
||||
this(HBaseConfiguration.create(), KeyValue.COMPARATOR);
|
||||
this(HBaseConfiguration.create(), CellComparator.COMPARATOR);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -112,7 +113,7 @@ public class DefaultMemStore implements MemStore {
|
|||
* @param c Comparator
|
||||
*/
|
||||
public DefaultMemStore(final Configuration conf,
|
||||
final KeyValue.KVComparator c) {
|
||||
final CellComparator c) {
|
||||
this.conf = conf;
|
||||
this.comparator = c;
|
||||
this.cellSet = new CellSkipListSet(c);
|
||||
|
|
|
@ -24,8 +24,8 @@ import java.util.List;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
|
@ -63,7 +63,7 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
|
||||
@Override
|
||||
protected void createComponents(
|
||||
Configuration conf, Store store, KVComparator kvComparator) throws IOException {
|
||||
Configuration conf, Store store, CellComparator kvComparator) throws IOException {
|
||||
String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
|
||||
try {
|
||||
compactor = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||
|
|
|
@ -29,8 +29,8 @@ 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.CellComparator;
|
||||
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.regionserver.compactions.CompactionConfiguration;
|
||||
|
||||
|
@ -45,7 +45,7 @@ import com.google.common.collect.Lists;
|
|||
class DefaultStoreFileManager implements StoreFileManager {
|
||||
private static final Log LOG = LogFactory.getLog(DefaultStoreFileManager.class);
|
||||
|
||||
private final KVComparator kvComparator;
|
||||
private final CellComparator kvComparator;
|
||||
private final CompactionConfiguration comConf;
|
||||
private final int blockingFileCount;
|
||||
|
||||
|
@ -55,7 +55,7 @@ class DefaultStoreFileManager implements StoreFileManager {
|
|||
*/
|
||||
private volatile ImmutableList<StoreFile> storefiles = null;
|
||||
|
||||
public DefaultStoreFileManager(KVComparator kvComparator, Configuration conf,
|
||||
public DefaultStoreFileManager(CellComparator kvComparator, Configuration conf,
|
||||
CompactionConfiguration comConf) {
|
||||
this.kvComparator = kvComparator;
|
||||
this.comConf = comConf;
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -45,7 +44,7 @@ class GetClosestRowBeforeTracker {
|
|||
private final long now;
|
||||
private final long oldestUnexpiredTs;
|
||||
private Cell candidate = null;
|
||||
private final KVComparator kvcomparator;
|
||||
private final CellComparator cellComparator;
|
||||
// Flag for whether we're doing getclosest on a metaregion.
|
||||
private final boolean metaregion;
|
||||
// Offset and length into targetkey demarking table name (if in a metaregion).
|
||||
|
@ -62,7 +61,7 @@ class GetClosestRowBeforeTracker {
|
|||
* @param ttl Time to live in ms for this Store
|
||||
* @param metaregion True if this is hbase:meta or -ROOT- region.
|
||||
*/
|
||||
GetClosestRowBeforeTracker(final KVComparator c, final KeyValue kv,
|
||||
GetClosestRowBeforeTracker(final CellComparator c, final KeyValue kv,
|
||||
final long ttl, final boolean metaregion) {
|
||||
super();
|
||||
this.metaregion = metaregion;
|
||||
|
@ -72,13 +71,13 @@ class GetClosestRowBeforeTracker {
|
|||
this.rowoffset = kv.getRowOffset();
|
||||
int l = -1;
|
||||
if (metaregion) {
|
||||
l = KeyValue.getDelimiter(kv.getRowArray(), rowoffset, kv.getRowLength(),
|
||||
l = Bytes.searchDelimiterIndex(kv.getRowArray(), rowoffset, kv.getRowLength(),
|
||||
HConstants.DELIMITER) - this.rowoffset;
|
||||
}
|
||||
this.tablenamePlusDelimiterLength = metaregion? l + 1: -1;
|
||||
this.now = System.currentTimeMillis();
|
||||
this.oldestUnexpiredTs = now - ttl;
|
||||
this.kvcomparator = c;
|
||||
this.cellComparator = c;
|
||||
this.deletes = new TreeMap<Cell, NavigableSet<Cell>>(new CellComparator.RowComparator());
|
||||
}
|
||||
|
||||
|
@ -89,7 +88,7 @@ class GetClosestRowBeforeTracker {
|
|||
private void addDelete(final Cell kv) {
|
||||
NavigableSet<Cell> rowdeletes = this.deletes.get(kv);
|
||||
if (rowdeletes == null) {
|
||||
rowdeletes = new TreeSet<Cell>(this.kvcomparator);
|
||||
rowdeletes = new TreeSet<Cell>(this.cellComparator);
|
||||
this.deletes.put(kv, rowdeletes);
|
||||
}
|
||||
rowdeletes.add(kv);
|
||||
|
@ -109,8 +108,8 @@ class GetClosestRowBeforeTracker {
|
|||
|
||||
boolean isBetterCandidate(final Cell contender) {
|
||||
return this.candidate == null ||
|
||||
(this.kvcomparator.compareRows(this.candidate, contender) < 0 &&
|
||||
this.kvcomparator.compareRows(contender, this.targetkey) <= 0);
|
||||
(this.cellComparator.compareRows(this.candidate, contender) < 0 &&
|
||||
this.cellComparator.compareRows(contender, this.targetkey) <= 0);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -231,7 +230,7 @@ class GetClosestRowBeforeTracker {
|
|||
* @return True if we went too far, past the target key.
|
||||
*/
|
||||
boolean isTooFar(final Cell kv, final Cell firstOnRow) {
|
||||
return this.kvcomparator.compareRows(kv, firstOnRow) > 0;
|
||||
return this.cellComparator.compareRows(kv, firstOnRow) > 0;
|
||||
}
|
||||
|
||||
boolean isTargetTable(final Cell kv) {
|
||||
|
|
|
@ -70,6 +70,7 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompoundConfiguration;
|
||||
|
@ -83,7 +84,6 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
|
@ -5199,7 +5199,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
private long readPt;
|
||||
private long maxResultSize;
|
||||
protected HRegion region;
|
||||
protected KVComparator comparator;
|
||||
protected CellComparator comparator;
|
||||
|
||||
@Override
|
||||
public HRegionInfo getRegionInfo() {
|
||||
|
@ -5716,6 +5716,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
protected boolean isStopRow(byte[] currentRow, int offset, short length) {
|
||||
return currentRow == null ||
|
||||
(stopRow != null &&
|
||||
// TODO : currentRow can be tracked as cell rather than byte[]
|
||||
comparator.compareRows(stopRow, 0, stopRow.length,
|
||||
currentRow, offset, length) <= isScan);
|
||||
}
|
||||
|
@ -7833,7 +7834,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
@Override
|
||||
public KVComparator getCellCompartor() {
|
||||
return this.getRegionInfo().isMetaRegion() ? KeyValue.META_COMPARATOR : KeyValue.COMPARATOR;
|
||||
public CellComparator getCellCompartor() {
|
||||
return this.getRegionInfo().isMetaRegion() ? CellComparator.META_COMPARATOR
|
||||
: CellComparator.COMPARATOR;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -591,8 +591,7 @@ public class HRegionFileSystem {
|
|||
if (lastKey == null) {
|
||||
return null;
|
||||
}
|
||||
if (f.getReader().getComparator().compareFlatKey(splitKey.getBuffer(),
|
||||
splitKey.getKeyOffset(), splitKey.getKeyLength(), lastKey, 0, lastKey.length) > 0) {
|
||||
if (f.getReader().getComparator().compare(splitKey, lastKey, 0, lastKey.length) > 0) {
|
||||
return null;
|
||||
}
|
||||
} else {
|
||||
|
@ -603,8 +602,7 @@ public class HRegionFileSystem {
|
|||
if (firstKey == null) {
|
||||
return null;
|
||||
}
|
||||
if (f.getReader().getComparator().compareFlatKey(splitKey.getBuffer(),
|
||||
splitKey.getKeyOffset(), splitKey.getKeyLength(), firstKey, 0, firstKey.length) < 0) {
|
||||
if (f.getReader().getComparator().compare(splitKey, firstKey, 0, firstKey.length) < 0) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
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.TagType;
|
||||
|
@ -178,7 +179,7 @@ public class HStore implements Store {
|
|||
private int bytesPerChecksum;
|
||||
|
||||
// Comparing KeyValues
|
||||
private final KeyValue.KVComparator comparator;
|
||||
private final CellComparator comparator;
|
||||
|
||||
final StoreEngine<?, ?, ?, ?> storeEngine;
|
||||
|
||||
|
@ -245,7 +246,7 @@ public class HStore implements Store {
|
|||
scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator);
|
||||
String className = conf.get(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName());
|
||||
this.memstore = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
|
||||
Configuration.class, KeyValue.KVComparator.class }, new Object[] { conf, this.comparator });
|
||||
Configuration.class, CellComparator.class }, new Object[] { conf, this.comparator });
|
||||
this.offPeakHours = OffPeakHours.getInstance(conf);
|
||||
|
||||
// Setting up cache configuration for this family
|
||||
|
@ -723,7 +724,7 @@ public class HStore implements Store {
|
|||
Preconditions.checkState(firstKey != null, "First key can not be null");
|
||||
byte[] lk = reader.getLastKey();
|
||||
Preconditions.checkState(lk != null, "Last key can not be null");
|
||||
byte[] lastKey = KeyValue.createKeyValueFromKey(lk).getRow();
|
||||
byte[] lastKey = KeyValueUtil.createKeyValueFromKey(lk).getRow();
|
||||
|
||||
LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
|
||||
" last=" + Bytes.toStringBinary(lastKey));
|
||||
|
@ -752,7 +753,7 @@ public class HStore implements Store {
|
|||
do {
|
||||
Cell cell = scanner.getKeyValue();
|
||||
if (prevCell != null) {
|
||||
if (CellComparator.compareRows(prevCell, cell) > 0) {
|
||||
if (comparator.compareRows(prevCell, cell) > 0) {
|
||||
throw new InvalidHFileException("Previous row is greater than"
|
||||
+ " current row: path=" + srcPath + " previous="
|
||||
+ CellUtil.getCellKeyAsString(prevCell) + " current="
|
||||
|
@ -1850,9 +1851,9 @@ public class HStore implements Store {
|
|||
// TODO: Cache these keys rather than make each time?
|
||||
byte [] fk = r.getFirstKey();
|
||||
if (fk == null) return false;
|
||||
KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
|
||||
KeyValue firstKV = KeyValueUtil.createKeyValueFromKey(fk, 0, fk.length);
|
||||
byte [] lk = r.getLastKey();
|
||||
KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
|
||||
KeyValue lastKV = KeyValueUtil.createKeyValueFromKey(lk, 0, lk.length);
|
||||
KeyValue firstOnRow = state.getTargetKey();
|
||||
if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
|
||||
// If last key in file is not of the target table, no candidates in this
|
||||
|
@ -2309,7 +2310,7 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
@Override
|
||||
public KeyValue.KVComparator getComparator() {
|
||||
public CellComparator getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.List;
|
|||
import java.util.PriorityQueue;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
|
||||
|
||||
|
@ -67,7 +67,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
|
|||
* @param comparator
|
||||
*/
|
||||
public KeyValueHeap(List<? extends KeyValueScanner> scanners,
|
||||
KVComparator comparator) throws IOException {
|
||||
CellComparator comparator) throws IOException {
|
||||
this(scanners, new KVScannerComparator(comparator));
|
||||
}
|
||||
|
||||
|
@ -166,12 +166,12 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
|
|||
}
|
||||
|
||||
protected static class KVScannerComparator implements Comparator<KeyValueScanner> {
|
||||
protected KVComparator kvComparator;
|
||||
protected CellComparator kvComparator;
|
||||
/**
|
||||
* Constructor
|
||||
* @param kvComparator
|
||||
*/
|
||||
public KVScannerComparator(KVComparator kvComparator) {
|
||||
public KVScannerComparator(CellComparator kvComparator) {
|
||||
this.kvComparator = kvComparator;
|
||||
}
|
||||
public int compare(KeyValueScanner left, KeyValueScanner right) {
|
||||
|
@ -204,7 +204,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
|
|||
/**
|
||||
* @return KVComparator
|
||||
*/
|
||||
public KVComparator getComparator() {
|
||||
public CellComparator getComparator() {
|
||||
return this.kvComparator;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,11 +23,11 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
|
@ -394,7 +394,7 @@ public interface Region extends ConfigurationObserver {
|
|||
RegionScanner getScanner(Scan scan) throws IOException;
|
||||
|
||||
/** The comparator to be used with the region */
|
||||
KVComparator getCellCompartor();
|
||||
CellComparator getCellCompartor();
|
||||
|
||||
/**
|
||||
* Perform one or more increment operations on a row.
|
||||
|
|
|
@ -24,8 +24,8 @@ import java.util.List;
|
|||
import org.apache.commons.lang.NotImplementedException;
|
||||
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.KeyValue.KVComparator;
|
||||
|
||||
/**
|
||||
* ReversedKeyValueHeap is used for supporting reversed scanning. Compared with
|
||||
|
@ -43,7 +43,7 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
|
|||
* @throws IOException
|
||||
*/
|
||||
public ReversedKeyValueHeap(List<? extends KeyValueScanner> scanners,
|
||||
KVComparator comparator) throws IOException {
|
||||
CellComparator comparator) throws IOException {
|
||||
super(scanners, new ReversedKVScannerComparator(comparator));
|
||||
}
|
||||
|
||||
|
@ -77,9 +77,7 @@ 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(),
|
||||
seekKey.getRowOffset(), seekKey.getRowLength()) < 0) {
|
||||
if (comparator.getComparator().compareRows(topKey, seekKey) < 0) {
|
||||
// Row of Top KeyValue is before Seek row.
|
||||
heap.add(scanner);
|
||||
current = pollRealKV();
|
||||
|
@ -162,7 +160,7 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
|
|||
* Constructor
|
||||
* @param kvComparator
|
||||
*/
|
||||
public ReversedKVScannerComparator(KVComparator kvComparator) {
|
||||
public ReversedKVScannerComparator(CellComparator kvComparator) {
|
||||
super(kvComparator);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,10 +24,9 @@ import java.util.NavigableSet;
|
|||
|
||||
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.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
||||
/**
|
||||
|
@ -63,7 +62,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
|
|||
|
||||
@Override
|
||||
protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
|
||||
KVComparator comparator) throws IOException {
|
||||
CellComparator comparator) throws IOException {
|
||||
// Combine all seeked scanners with a heap
|
||||
heap = new ReversedKeyValueHeap(scanners, comparator);
|
||||
}
|
||||
|
@ -100,11 +99,11 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
|
|||
|
||||
@Override
|
||||
protected void checkScanOrder(Cell prevKV, Cell kv,
|
||||
KeyValue.KVComparator comparator) throws IOException {
|
||||
CellComparator comparator) throws IOException {
|
||||
// Check that the heap gives us KVs in an increasing order for same row and
|
||||
// decreasing order for different rows.
|
||||
assert prevKV == null || comparator == null || comparator.compareRows(kv, prevKV) < 0
|
||||
|| (comparator.matchingRows(kv, prevKV) && comparator.compare(kv,
|
||||
|| (CellUtil.matchingRows(kv, prevKV) && comparator.compare(kv,
|
||||
prevKV) >= 0) : "Key " + prevKV
|
||||
+ " followed by a " + "error order key " + kv + " in cf " + store
|
||||
+ " in reversed scan";
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
|
||||
|
@ -36,7 +36,7 @@ public class ScanInfo {
|
|||
private long ttl;
|
||||
private KeepDeletedCells keepDeletedCells;
|
||||
private long timeToPurgeDeletes;
|
||||
private KVComparator comparator;
|
||||
private CellComparator comparator;
|
||||
|
||||
public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
|
||||
+ (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
|
||||
|
@ -50,7 +50,7 @@ public class ScanInfo {
|
|||
* @param comparator The store's comparator
|
||||
*/
|
||||
public ScanInfo(final HColumnDescriptor family, final long ttl, final long timeToPurgeDeletes,
|
||||
final KVComparator comparator) {
|
||||
final CellComparator comparator) {
|
||||
this(family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl, family
|
||||
.getKeepDeletedCells(), timeToPurgeDeletes, comparator);
|
||||
}
|
||||
|
@ -67,7 +67,7 @@ public class ScanInfo {
|
|||
*/
|
||||
public ScanInfo(final byte[] family, final int minVersions, final int maxVersions,
|
||||
final long ttl, final KeepDeletedCells keepDeletedCells, final long timeToPurgeDeletes,
|
||||
final KVComparator comparator) {
|
||||
final CellComparator comparator) {
|
||||
this.family = family;
|
||||
this.minVersions = minVersions;
|
||||
this.maxVersions = maxVersions;
|
||||
|
@ -101,7 +101,7 @@ public class ScanInfo {
|
|||
return timeToPurgeDeletes;
|
||||
}
|
||||
|
||||
public KVComparator getComparator() {
|
||||
public CellComparator getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,10 +25,10 @@ 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.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
|
@ -86,7 +86,7 @@ public class ScanQueryMatcher {
|
|||
private final Cell startKey;
|
||||
|
||||
/** Row comparator for the region this query is for */
|
||||
private final KeyValue.KVComparator rowComparator;
|
||||
private final CellComparator rowComparator;
|
||||
|
||||
/* row is not private for tests */
|
||||
/** Row the query is on */
|
||||
|
@ -278,8 +278,7 @@ 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());
|
||||
int ret = -(this.rowComparator.compareRows(cell, row, this.rowOffset, this.rowLength));
|
||||
if (!this.isReversed) {
|
||||
if (ret <= -1) {
|
||||
return MatchCode.DONE;
|
||||
|
@ -495,16 +494,14 @@ public class ScanQueryMatcher {
|
|||
|
||||
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;
|
||||
|
@ -584,32 +581,25 @@ 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());
|
||||
return rowComparator.compareKeyBasedOnColHint(nextIndexed, kv, 0, 0, null, 0, 0,
|
||||
HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param nextIndexed the key of the next entry in the block index (if any)
|
||||
* @param kv The Cell we're using to calculate the seek key
|
||||
* @param currentCell The Cell we're using to calculate the seek key
|
||||
* @return result of the compare between the indexed key and the key portion of the passed cell
|
||||
*/
|
||||
public int compareKeyForNextColumn(Cell nextIndexed, Cell kv) {
|
||||
public int compareKeyForNextColumn(Cell nextIndexed, Cell currentCell) {
|
||||
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());
|
||||
return rowComparator.compareKeyBasedOnColHint(nextIndexed, currentCell, 0, 0, null, 0, 0,
|
||||
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());
|
||||
return rowComparator.compareKeyBasedOnColHint(nextIndexed, currentCell,
|
||||
currentCell.getFamilyOffset(), currentCell.getFamilyLength(), nextColumn.getBuffer(),
|
||||
nextColumn.getOffset(), nextColumn.getLength(), HConstants.LATEST_TIMESTAMP,
|
||||
Type.Maximum.getCode());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -27,10 +27,10 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
|
||||
|
@ -59,7 +59,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
int NO_PRIORITY = Integer.MIN_VALUE;
|
||||
|
||||
// General Accessors
|
||||
KeyValue.KVComparator getComparator();
|
||||
CellComparator getComparator();
|
||||
|
||||
Collection<StoreFile> getStorefiles();
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||
|
@ -97,10 +97,10 @@ public abstract class StoreEngine<SF extends StoreFlusher,
|
|||
* Create the StoreEngine's components.
|
||||
*/
|
||||
protected abstract void createComponents(
|
||||
Configuration conf, Store store, KVComparator kvComparator) throws IOException;
|
||||
Configuration conf, Store store, CellComparator kvComparator) throws IOException;
|
||||
|
||||
private void createComponentsOnce(
|
||||
Configuration conf, Store store, KVComparator kvComparator) throws IOException {
|
||||
Configuration conf, Store store, CellComparator kvComparator) throws IOException {
|
||||
assert compactor == null && compactionPolicy == null
|
||||
&& storeFileManager == null && storeFlusher == null;
|
||||
createComponents(conf, store, kvComparator);
|
||||
|
@ -117,7 +117,7 @@ public abstract class StoreEngine<SF extends StoreFlusher,
|
|||
* @return StoreEngine to use.
|
||||
*/
|
||||
public static StoreEngine<?, ?, ?, ?> create(
|
||||
Store store, Configuration conf, KVComparator kvComparator) throws IOException {
|
||||
Store store, Configuration conf, CellComparator kvComparator) throws IOException {
|
||||
String className = conf.get(STORE_ENGINE_CLASS_KEY, DEFAULT_STORE_ENGINE_CLASS.getName());
|
||||
try {
|
||||
StoreEngine<?,?,?,?> se = ReflectionUtils.instantiateWithCustomCtor(
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -538,7 +538,7 @@ public class StoreFile {
|
|||
private final CacheConfig cacheConf;
|
||||
private final FileSystem fs;
|
||||
|
||||
private KeyValue.KVComparator comparator = KeyValue.COMPARATOR;
|
||||
private CellComparator comparator = CellComparator.COMPARATOR;
|
||||
private BloomType bloomType = BloomType.NONE;
|
||||
private long maxKeyCount = 0;
|
||||
private Path dir;
|
||||
|
@ -585,7 +585,7 @@ public class StoreFile {
|
|||
return this;
|
||||
}
|
||||
|
||||
public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
|
||||
public WriterBuilder withComparator(CellComparator comparator) {
|
||||
Preconditions.checkNotNull(comparator);
|
||||
this.comparator = comparator;
|
||||
return this;
|
||||
|
@ -637,7 +637,7 @@ public class StoreFile {
|
|||
}
|
||||
|
||||
if (comparator == null) {
|
||||
comparator = KeyValue.COMPARATOR;
|
||||
comparator = CellComparator.COMPARATOR;
|
||||
}
|
||||
return new Writer(fs, filePath,
|
||||
conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext);
|
||||
|
@ -670,7 +670,7 @@ public class StoreFile {
|
|||
* @return The split point row, or null if splitting is not possible, or reader is null.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
byte[] getFileSplitPoint(KVComparator comparator) throws IOException {
|
||||
byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
|
||||
if (this.reader == null) {
|
||||
LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
|
||||
return null;
|
||||
|
@ -680,11 +680,11 @@ public class StoreFile {
|
|||
// the row we want to split on as midkey.
|
||||
byte [] midkey = this.reader.midkey();
|
||||
if (midkey != null) {
|
||||
KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
|
||||
KeyValue mk = KeyValueUtil.createKeyValueFromKey(midkey, 0, midkey.length);
|
||||
byte [] fk = this.reader.getFirstKey();
|
||||
KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
|
||||
KeyValue firstKey = KeyValueUtil.createKeyValueFromKey(fk, 0, fk.length);
|
||||
byte [] lk = this.reader.getLastKey();
|
||||
KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
|
||||
KeyValue lastKey = KeyValueUtil.createKeyValueFromKey(lk, 0, lk.length);
|
||||
// if the midkey is the same as the first or last keys, we cannot (ever) split this region.
|
||||
if (comparator.compareRows(mk, firstKey) == 0 || comparator.compareRows(mk, lastKey) == 0) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -707,7 +707,7 @@ public class StoreFile {
|
|||
private final BloomType bloomType;
|
||||
private byte[] lastBloomKey;
|
||||
private int lastBloomKeyOffset, lastBloomKeyLen;
|
||||
private KVComparator kvComparator;
|
||||
private CellComparator kvComparator;
|
||||
private Cell lastCell = null;
|
||||
private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
|
||||
private Cell lastDeleteFamilyCell = null;
|
||||
|
@ -726,6 +726,7 @@ public class StoreFile {
|
|||
boolean isTimeRangeTrackerSet = false;
|
||||
|
||||
protected HFile.Writer writer;
|
||||
private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
|
||||
|
||||
/**
|
||||
* Creates an HFile.Writer that also write helpful meta data.
|
||||
|
@ -743,7 +744,7 @@ public class StoreFile {
|
|||
private Writer(FileSystem fs, Path path,
|
||||
final Configuration conf,
|
||||
CacheConfig cacheConf,
|
||||
final KVComparator comparator, BloomType bloomType, long maxKeys,
|
||||
final CellComparator comparator, BloomType bloomType, long maxKeys,
|
||||
InetSocketAddress[] favoredNodes, HFileContext fileContext)
|
||||
throws IOException {
|
||||
writer = HFile.getWriterFactory(conf, cacheConf)
|
||||
|
@ -761,6 +762,9 @@ public class StoreFile {
|
|||
|
||||
if (generalBloomFilterWriter != null) {
|
||||
this.bloomType = bloomType;
|
||||
if(this.bloomType == BloomType.ROWCOL) {
|
||||
lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue();
|
||||
}
|
||||
if (LOG.isTraceEnabled()) LOG.trace("Bloom filter type for " + path + ": " +
|
||||
this.bloomType + ", " + generalBloomFilterWriter.getClass().getSimpleName());
|
||||
} else {
|
||||
|
@ -838,10 +842,10 @@ public class StoreFile {
|
|||
if (this.lastCell != null) {
|
||||
switch(bloomType) {
|
||||
case ROW:
|
||||
newKey = ! kvComparator.matchingRows(cell, lastCell);
|
||||
newKey = ! CellUtil.matchingRows(cell, lastCell);
|
||||
break;
|
||||
case ROWCOL:
|
||||
newKey = ! kvComparator.matchingRowColumn(cell, lastCell);
|
||||
newKey = ! CellUtil.matchingRowColumn(cell, lastCell);
|
||||
break;
|
||||
case NONE:
|
||||
newKey = false;
|
||||
|
@ -885,17 +889,17 @@ public class StoreFile {
|
|||
}
|
||||
generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
|
||||
if (lastBloomKey != null) {
|
||||
int res = 0;
|
||||
boolean res = false;
|
||||
// hbase:meta does not have blooms. So we need not have special interpretation
|
||||
// of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
|
||||
if (bloomType == BloomType.ROW) {
|
||||
res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen,
|
||||
lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen);
|
||||
lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen) <= 0;
|
||||
} else {
|
||||
res = KeyValue.COMPARATOR.compareFlatKey(bloomKey,
|
||||
bloomKeyOffset, bloomKeyLen, lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen);
|
||||
res = (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, bloomKey,
|
||||
bloomKeyOffset, bloomKeyLen) >= 0);
|
||||
}
|
||||
if (res <= 0) {
|
||||
if (res) {
|
||||
throw new IOException("Non-increasing Bloom keys: "
|
||||
+ Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after "
|
||||
+ Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen));
|
||||
|
@ -904,6 +908,9 @@ public class StoreFile {
|
|||
lastBloomKey = bloomKey;
|
||||
lastBloomKeyOffset = bloomKeyOffset;
|
||||
lastBloomKeyLen = bloomKeyLen;
|
||||
if (bloomType == BloomType.ROWCOL) {
|
||||
lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen);
|
||||
}
|
||||
this.lastCell = cell;
|
||||
}
|
||||
}
|
||||
|
@ -922,7 +929,7 @@ public class StoreFile {
|
|||
if (lastDeleteFamilyCell != null) {
|
||||
// hbase:meta does not have blooms. So we need not have special interpretation
|
||||
// of the hbase:meta cells
|
||||
newKey = !kvComparator.matchingRows(cell, lastDeleteFamilyCell);
|
||||
newKey = !CellUtil.matchingRows(cell, lastDeleteFamilyCell);
|
||||
}
|
||||
if (newKey) {
|
||||
this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(),
|
||||
|
@ -1039,6 +1046,7 @@ public class StoreFile {
|
|||
private byte[] lastBloomKey;
|
||||
private long deleteFamilyCnt = -1;
|
||||
private boolean bulkLoadResult = false;
|
||||
private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
|
||||
|
||||
public Reader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
|
||||
throws IOException {
|
||||
|
@ -1059,7 +1067,7 @@ public class StoreFile {
|
|||
this.reader = null;
|
||||
}
|
||||
|
||||
public KVComparator getComparator() {
|
||||
public CellComparator getComparator() {
|
||||
return reader.getComparator();
|
||||
}
|
||||
|
||||
|
@ -1296,7 +1304,9 @@ public class StoreFile {
|
|||
if (bloomFilterType == BloomType.ROW) {
|
||||
keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
|
||||
} else {
|
||||
keyIsAfterLast = (KeyValue.COMPARATOR.compareFlatKey(key, lastBloomKey)) > 0;
|
||||
// TODO : Convert key to Cell so that we could use compare(Cell, Cell)
|
||||
keyIsAfterLast = (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, key, 0,
|
||||
key.length)) < 0;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1307,10 +1317,11 @@ public class StoreFile {
|
|||
// required looking only for a row bloom.
|
||||
byte[] rowBloomKey = bloomFilter.createBloomKey(row, rowOffset, rowLen,
|
||||
null, 0, 0);
|
||||
|
||||
// hbase:meta does not have blooms. So we need not have special interpretation
|
||||
// of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
|
||||
if (keyIsAfterLast
|
||||
&& KeyValue.COMPARATOR.compareFlatKey(rowBloomKey,
|
||||
lastBloomKey) > 0) {
|
||||
&& (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, rowBloomKey, 0,
|
||||
rowBloomKey.length)) < 0) {
|
||||
exists = false;
|
||||
} else {
|
||||
exists =
|
||||
|
@ -1357,12 +1368,16 @@ public class StoreFile {
|
|||
KeyValue largestScanKeyValue = scan.isReversed() ? KeyValueUtil
|
||||
.createLastOnRow(scan.getStartRow()) : KeyValueUtil.createLastOnRow(scan
|
||||
.getStopRow());
|
||||
boolean nonOverLapping = (getComparator().compareFlatKey(
|
||||
this.getFirstKey(), largestScanKeyValue.getKey()) > 0 && !Bytes
|
||||
// TODO this is in hot path? Optimize and avoid 2 extra object creations.
|
||||
KeyValue.KeyOnlyKeyValue firstKeyKV =
|
||||
new KeyValue.KeyOnlyKeyValue(this.getFirstKey(), 0, this.getFirstKey().length);
|
||||
KeyValue.KeyOnlyKeyValue lastKeyKV =
|
||||
new KeyValue.KeyOnlyKeyValue(this.getLastKey(), 0, this.getLastKey().length);
|
||||
boolean nonOverLapping = ((getComparator().compare(firstKeyKV, largestScanKeyValue)) > 0
|
||||
&& !Bytes
|
||||
.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
|
||||
HConstants.EMPTY_END_ROW))
|
||||
|| getComparator().compareFlatKey(this.getLastKey(),
|
||||
smallestScanKeyValue.getKey()) < 0;
|
||||
|| (getComparator().compare(lastKeyKV, smallestScanKeyValue)) < 0;
|
||||
return !nonOverLapping;
|
||||
}
|
||||
|
||||
|
@ -1375,6 +1390,9 @@ public class StoreFile {
|
|||
}
|
||||
|
||||
lastBloomKey = fi.get(LAST_BLOOM_KEY);
|
||||
if(bloomFilterType == BloomType.ROWCOL) {
|
||||
lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
|
||||
}
|
||||
byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
|
||||
if (cnt != null) {
|
||||
deleteFamilyCnt = Bytes.toLong(cnt);
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
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.KeyValue;
|
||||
|
@ -215,9 +216,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
hfs.next();
|
||||
setCurrentCell(hfs.getKeyValue());
|
||||
if (this.stopSkippingKVsIfNextRow
|
||||
&& getComparator().compareRows(cur.getRowArray(), cur.getRowOffset(),
|
||||
cur.getRowLength(), startKV.getRowArray(), startKV.getRowOffset(),
|
||||
startKV.getRowLength()) > 0) {
|
||||
&& getComparator().compareRows(cur, startKV) > 0) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -369,7 +368,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
return reader;
|
||||
}
|
||||
|
||||
KeyValue.KVComparator getComparator() {
|
||||
CellComparator getComparator() {
|
||||
return reader.getComparator();
|
||||
}
|
||||
|
||||
|
@ -476,9 +475,7 @@ 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(),
|
||||
key.getRowLength()) > 0) {
|
||||
|| getComparator().compareRows(cur, key) > 0) {
|
||||
return seekToPreviousRow(key);
|
||||
}
|
||||
return true;
|
||||
|
|
|
@ -31,11 +31,11 @@ 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.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
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.client.IsolationLevel;
|
||||
|
@ -373,7 +373,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
}
|
||||
|
||||
protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
|
||||
KVComparator comparator) throws IOException {
|
||||
CellComparator comparator) throws IOException {
|
||||
// Combine all seeked scanners with a heap
|
||||
heap = new KeyValueHeap(scanners, comparator);
|
||||
}
|
||||
|
@ -523,7 +523,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
Cell cell;
|
||||
|
||||
// Only do a sanity-check if store and comparator are available.
|
||||
KeyValue.KVComparator comparator =
|
||||
CellComparator comparator =
|
||||
store != null ? store.getComparator() : null;
|
||||
|
||||
int count = 0;
|
||||
|
@ -781,7 +781,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
* @throws IOException
|
||||
*/
|
||||
protected void checkScanOrder(Cell prevKV, Cell kv,
|
||||
KeyValue.KVComparator comparator) throws IOException {
|
||||
CellComparator comparator) throws IOException {
|
||||
// Check that the heap gives us KVs in an increasing order.
|
||||
assert prevKV == null || comparator == null
|
||||
|| comparator.compare(prevKV, kv) <= 0 : "Key " + prevKV
|
||||
|
|
|
@ -27,7 +27,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -40,7 +41,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
|
||||
/** Factory that is used to produce single StoreFile.Writer-s */
|
||||
protected WriterFactory writerFactory;
|
||||
protected KVComparator comparator;
|
||||
protected CellComparator comparator;
|
||||
|
||||
protected List<StoreFile.Writer> existingWriters;
|
||||
protected List<byte[]> boundaries;
|
||||
|
@ -60,7 +61,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
* @param factory Factory used to produce individual file writers.
|
||||
* @param comparator Comparator used to compare rows.
|
||||
*/
|
||||
public void init(StoreScanner sourceScanner, WriterFactory factory, KVComparator comparator)
|
||||
public void init(StoreScanner sourceScanner, WriterFactory factory, CellComparator comparator)
|
||||
throws IOException {
|
||||
this.writerFactory = factory;
|
||||
this.sourceScanner = sourceScanner;
|
||||
|
@ -111,16 +112,15 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
/**
|
||||
* Subclasses can call this method to make sure the first KV is within multi-writer range.
|
||||
* @param left The left boundary of the writer.
|
||||
* @param row The row to check.
|
||||
* @param rowOffset Offset for row.
|
||||
* @param rowLength Length for row.
|
||||
* @param cell The cell whose row has to be checked.
|
||||
*/
|
||||
protected void sanityCheckLeft(
|
||||
byte[] left, byte[] row, int rowOffset, int rowLength) throws IOException {
|
||||
byte[] left, Cell cell) throws IOException {
|
||||
if (StripeStoreFileManager.OPEN_KEY != left &&
|
||||
comparator.compareRows(row, rowOffset, rowLength, left, 0, left.length) < 0) {
|
||||
comparator.compareRows(cell, left, 0, left.length) < 0) {
|
||||
String error = "The first row is lower than the left boundary of [" + Bytes.toString(left)
|
||||
+ "]: [" + Bytes.toString(row, rowOffset, rowLength) + "]";
|
||||
+ "]: [" + Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())
|
||||
+ "]";
|
||||
LOG.error(error);
|
||||
throw new IOException(error);
|
||||
}
|
||||
|
@ -129,16 +129,14 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
/**
|
||||
* Subclasses can call this method to make sure the last KV is within multi-writer range.
|
||||
* @param right The right boundary of the writer.
|
||||
* @param row The row to check.
|
||||
* @param rowOffset Offset for row.
|
||||
* @param rowLength Length for row.
|
||||
*/
|
||||
protected void sanityCheckRight(
|
||||
byte[] right, byte[] row, int rowOffset, int rowLength) throws IOException {
|
||||
byte[] right, Cell cell) throws IOException {
|
||||
if (StripeStoreFileManager.OPEN_KEY != right &&
|
||||
comparator.compareRows(row, rowOffset, rowLength, right, 0, right.length) >= 0) {
|
||||
comparator.compareRows(cell, right, 0, right.length) >= 0) {
|
||||
String error = "The last row is higher or equal than the right boundary of ["
|
||||
+ Bytes.toString(right) + "]: [" + Bytes.toString(row, rowOffset, rowLength) + "]";
|
||||
+ Bytes.toString(right) + "]: ["
|
||||
+ Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()) + "]";
|
||||
LOG.error(error);
|
||||
throw new IOException(error);
|
||||
}
|
||||
|
@ -197,7 +195,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
if (currentWriter == null && existingWriters.isEmpty()) {
|
||||
// First append ever, do a sanity check.
|
||||
sanityCheckLeft(this.boundaries.get(0),
|
||||
cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
cell);
|
||||
}
|
||||
prepareWriterFor(cell);
|
||||
currentWriter.append(cell);
|
||||
|
@ -207,7 +205,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
|
||||
private boolean isCellAfterCurrentWriter(Cell cell) {
|
||||
return ((currentWriterEndKey != StripeStoreFileManager.OPEN_KEY) &&
|
||||
(comparator.compareRows(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
(comparator.compareRows(cell,
|
||||
currentWriterEndKey, 0, currentWriterEndKey.length) >= 0));
|
||||
}
|
||||
|
||||
|
@ -219,7 +217,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
}
|
||||
if (lastCell != null) {
|
||||
sanityCheckRight(boundaries.get(boundaries.size() - 1),
|
||||
lastCell.getRowArray(), lastCell.getRowOffset(), lastCell.getRowLength());
|
||||
lastCell);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -326,10 +324,10 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
boolean doCreateWriter = false;
|
||||
if (currentWriter == null) {
|
||||
// First append ever, do a sanity check.
|
||||
sanityCheckLeft(left, cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
sanityCheckLeft(left, cell);
|
||||
doCreateWriter = true;
|
||||
} else if (lastRowInCurrentWriter != null
|
||||
&& !comparator.matchingRows(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
&& !CellUtil.matchingRow(cell,
|
||||
lastRowInCurrentWriter, 0, lastRowInCurrentWriter.length)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Stopping to use a writer after [" + Bytes.toString(lastRowInCurrentWriter)
|
||||
|
@ -382,7 +380,7 @@ public abstract class StripeMultiFileWriter implements Compactor.CellSink {
|
|||
}
|
||||
if (lastCell != null) {
|
||||
sanityCheckRight(
|
||||
right, lastCell.getRowArray(), lastCell.getRowOffset(), lastCell.getRowLength());
|
||||
right, lastCell);
|
||||
}
|
||||
|
||||
// When expired stripes were going to be merged into one, and if no writer was created during
|
||||
|
|
|
@ -26,8 +26,8 @@ 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.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
|
||||
|
@ -57,7 +57,7 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
|
|||
|
||||
@Override
|
||||
protected void createComponents(
|
||||
Configuration conf, Store store, KVComparator comparator) throws IOException {
|
||||
Configuration conf, Store store, CellComparator comparator) throws IOException {
|
||||
this.config = new StripeStoreConfig(conf, store);
|
||||
this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
|
||||
this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
|
||||
|
|
|
@ -33,9 +33,9 @@ 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.CellComparator;
|
||||
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.regionserver.compactions.StripeCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -115,14 +115,14 @@ public class StripeStoreFileManager
|
|||
* we use it to compare by reference when we read from the map. */
|
||||
private static final byte[] INVALID_KEY_IN_MAP = new byte[0];
|
||||
|
||||
private final KVComparator kvComparator;
|
||||
private final CellComparator cellComparator;
|
||||
private StripeStoreConfig config;
|
||||
|
||||
private final int blockingFileCount;
|
||||
|
||||
public StripeStoreFileManager(
|
||||
KVComparator kvComparator, Configuration conf, StripeStoreConfig config) {
|
||||
this.kvComparator = kvComparator;
|
||||
CellComparator kvComparator, Configuration conf, StripeStoreConfig config) {
|
||||
this.cellComparator = kvComparator;
|
||||
this.config = config;
|
||||
this.blockingFileCount = conf.getInt(
|
||||
HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT);
|
||||
|
@ -256,7 +256,7 @@ public class StripeStoreFileManager
|
|||
+ newRatio + " configured ratio " + config.getMaxSplitImbalance());
|
||||
// Ok, we may get better ratio, get it.
|
||||
return StoreUtils.getLargestFile(state.stripeFiles.get(
|
||||
isRightLarger ? rightIndex : leftIndex)).getFileSplitPoint(this.kvComparator);
|
||||
isRightLarger ? rightIndex : leftIndex)).getFileSplitPoint(this.cellComparator);
|
||||
}
|
||||
|
||||
private byte[] getSplitPointFromAllFiles() throws IOException {
|
||||
|
@ -264,7 +264,7 @@ public class StripeStoreFileManager
|
|||
sfs.addSublist(state.level0Files);
|
||||
sfs.addAllSublists(state.stripeFiles);
|
||||
if (sfs.isEmpty()) return null;
|
||||
return StoreUtils.getLargestFile(sfs).getFileSplitPoint(this.kvComparator);
|
||||
return StoreUtils.getLargestFile(sfs).getFileSplitPoint(this.cellComparator);
|
||||
}
|
||||
|
||||
private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) {
|
||||
|
@ -512,7 +512,7 @@ public class StripeStoreFileManager
|
|||
* Compare two keys for equality.
|
||||
*/
|
||||
private final boolean rowEquals(byte[] k1, byte[] k2) {
|
||||
return kvComparator.matchingRows(k1, 0, k1.length, k2, 0, k2.length);
|
||||
return Bytes.equals(k1, 0, k1.length, k2, 0, k2.length);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -520,7 +520,7 @@ public class StripeStoreFileManager
|
|||
*/
|
||||
private final int nonOpenRowCompare(byte[] k1, byte[] k2) {
|
||||
assert !isOpen(k1) && !isOpen(k2);
|
||||
return kvComparator.compareRows(k1, 0, k1.length, k2, 0, k2.length);
|
||||
return cellComparator.compareRows(k1, 0, k1.length, k2, 0, k2.length);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -48,7 +48,8 @@ import com.google.protobuf.ByteString;
|
|||
* This codec is used at server side for writing cells to WAL as well as for sending edits
|
||||
* as part of the distributed splitting process.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC,
|
||||
HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
|
||||
public class WALCellCodec implements Codec {
|
||||
/** Configuration key for the class to use when encoding cells in the WAL */
|
||||
public static final String WAL_CELL_CODEC_CLASS_KEY = "hbase.regionserver.wal.codec";
|
||||
|
|
|
@ -185,7 +185,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
if (compressionContext != null) {
|
||||
this.add(KeyValueCompression.readKV(in, compressionContext));
|
||||
} else {
|
||||
this.add(KeyValue.create(in));
|
||||
this.add(KeyValueUtil.create(in));
|
||||
}
|
||||
}
|
||||
int numFamilies = in.readInt();
|
||||
|
@ -202,7 +202,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
} else {
|
||||
// this is an old style WAL entry. The int that we just
|
||||
// read is actually the length of a single KeyValue
|
||||
this.add(KeyValue.create(versionOrLength, in));
|
||||
this.add(KeyValueUtil.create(versionOrLength, in));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -218,7 +218,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
if (compressionContext != null) {
|
||||
KeyValueCompression.writeKV(out, kv, compressionContext);
|
||||
} else{
|
||||
KeyValue.write(kv, out);
|
||||
KeyValueUtil.write(kv, out);
|
||||
}
|
||||
}
|
||||
if (scopes == null) {
|
||||
|
|
|
@ -53,7 +53,7 @@ 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, ByteBuffer bloom);
|
||||
|
||||
/**
|
||||
* @return true if this Bloom filter can automatically load its data
|
||||
|
|
|
@ -26,7 +26,7 @@ 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.KeyValue;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
|
@ -103,6 +103,7 @@ public final class BloomFilterFactory {
|
|||
// This is only possible in a version 1 HFile. We are ignoring the
|
||||
// passed comparator because raw byte comparators are always used
|
||||
// in version 1 Bloom filters.
|
||||
// TODO:Remove this code - use only CompoundBloomFilter
|
||||
return new ByteBloomFilter(meta);
|
||||
|
||||
case CompoundBloomFilterBase.VERSION:
|
||||
|
@ -199,7 +200,7 @@ public final class BloomFilterFactory {
|
|||
// In case of compound Bloom filters we ignore the maxKeys hint.
|
||||
CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf),
|
||||
err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(),
|
||||
bloomType == BloomType.ROWCOL ? KeyValue.COMPARATOR : null);
|
||||
bloomType == BloomType.ROWCOL ? CellComparator.COMPARATOR : null);
|
||||
writer.addInlineBlockWriter(bloomWriter);
|
||||
return bloomWriter;
|
||||
}
|
||||
|
|
|
@ -76,6 +76,7 @@ import org.apache.hadoop.io.Writable;
|
|||
* Space/Time Trade-Offs in Hash Coding with Allowable Errors</a>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
// TODO : Remove this ByteBloomFilter as an instance of BloomFilter
|
||||
public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
|
||||
|
||||
/** Current file format version */
|
||||
|
@ -399,8 +400,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(byte[] buf, int offset, int length,
|
||||
ByteBuffer theBloom) {
|
||||
public boolean contains(byte[] buf, int offset, int length, ByteBuffer 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.
|
||||
|
@ -505,7 +505,7 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter {
|
|||
int newMaxKeys = this.maxKeys;
|
||||
|
||||
// while exponentially smaller & folding is lossless
|
||||
while ( (newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
|
||||
while ((newByteSize & 1) == 0 && newMaxKeys > (this.keyCount<<1) ) {
|
||||
pieces <<= 1;
|
||||
newByteSize >>= 1;
|
||||
newMaxKeys >>= 1;
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.SortedSet;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
|
||||
|
||||
/**
|
||||
|
@ -36,34 +36,34 @@ import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
|
|||
@InterfaceAudience.Private
|
||||
public class CollectionBackedScanner extends NonReversedNonLazyKeyValueScanner {
|
||||
final private Iterable<Cell> data;
|
||||
final KeyValue.KVComparator comparator;
|
||||
final CellComparator comparator;
|
||||
private Iterator<Cell> iter;
|
||||
private Cell current;
|
||||
|
||||
public CollectionBackedScanner(SortedSet<Cell> set) {
|
||||
this(set, KeyValue.COMPARATOR);
|
||||
this(set, CellComparator.COMPARATOR);
|
||||
}
|
||||
|
||||
public CollectionBackedScanner(SortedSet<Cell> set,
|
||||
KeyValue.KVComparator comparator) {
|
||||
CellComparator comparator) {
|
||||
this.comparator = comparator;
|
||||
data = set;
|
||||
init();
|
||||
}
|
||||
|
||||
public CollectionBackedScanner(List<Cell> list) {
|
||||
this(list, KeyValue.COMPARATOR);
|
||||
this(list, CellComparator.COMPARATOR);
|
||||
}
|
||||
|
||||
public CollectionBackedScanner(List<Cell> list,
|
||||
KeyValue.KVComparator comparator) {
|
||||
CellComparator comparator) {
|
||||
Collections.sort(list, comparator);
|
||||
this.comparator = comparator;
|
||||
data = list;
|
||||
init();
|
||||
}
|
||||
|
||||
public CollectionBackedScanner(KeyValue.KVComparator comparator,
|
||||
public CollectionBackedScanner(CellComparator comparator,
|
||||
Cell... array) {
|
||||
this.comparator = comparator;
|
||||
|
||||
|
|
|
@ -81,20 +81,19 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
|
|||
if (hash == null) {
|
||||
throw new IllegalArgumentException("Invalid hash type: " + hashType);
|
||||
}
|
||||
|
||||
// We will pass null for ROW block
|
||||
index = new HFileBlockIndex.BlockIndexReader(comparator, 1);
|
||||
index.readRootIndex(meta, numChunks);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(byte[] key, int keyOffset, int keyLength,
|
||||
ByteBuffer bloom) {
|
||||
public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuffer 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);
|
||||
keyLength, comparator);
|
||||
if (block < 0) {
|
||||
result = false; // This key is not in the file.
|
||||
} else {
|
||||
|
|
|
@ -20,8 +20,9 @@
|
|||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
|
@ -51,7 +52,7 @@ public class CompoundBloomFilterBase implements BloomFilterBase {
|
|||
/** Hash function type to use, as defined in {@link Hash} */
|
||||
protected int hashType;
|
||||
/** Comparator used to compare Bloom filter keys */
|
||||
protected KVComparator comparator;
|
||||
protected CellComparator comparator;
|
||||
|
||||
@Override
|
||||
public long getMaxKeys() {
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.Queue;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
|
||||
import org.apache.hadoop.hbase.io.hfile.InlineBlockWriter;
|
||||
|
@ -89,7 +89,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
|
|||
*/
|
||||
public CompoundBloomFilterWriter(int chunkByteSizeHint, float errorRate,
|
||||
int hashType, int maxFold, boolean cacheOnWrite,
|
||||
KVComparator comparator) {
|
||||
CellComparator comparator) {
|
||||
chunkByteSize = ByteBloomFilter.computeFoldableByteSize(
|
||||
chunkByteSizeHint * 8L, maxFold);
|
||||
|
||||
|
|
|
@ -139,7 +139,7 @@ public class CompressionTest {
|
|||
scanner.seekTo(); // position to the start of file
|
||||
// Scanner does not do Cells yet. Do below for now till fixed.
|
||||
cc = scanner.getKeyValue();
|
||||
if (CellComparator.compareRows(c, cc) != 0) {
|
||||
if (CellComparator.COMPARATOR.compareRows(c, cc) != 0) {
|
||||
throw new Exception("Read back incorrect result: " + c.toString() + " vs " + cc.toString());
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
|
@ -860,9 +861,9 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
CacheConfig cacheConf = new CacheConfig(getConf());
|
||||
hf = HFile.createReader(fs, hfile.getPath(), cacheConf, getConf());
|
||||
hf.loadFileInfo();
|
||||
KeyValue startKv = KeyValue.createKeyValueFromKey(hf.getFirstKey());
|
||||
KeyValue startKv = KeyValueUtil.createKeyValueFromKey(hf.getFirstKey());
|
||||
start = startKv.getRow();
|
||||
KeyValue endKv = KeyValue.createKeyValueFromKey(hf.getLastKey());
|
||||
KeyValue endKv = KeyValueUtil.createKeyValueFromKey(hf.getLastKey());
|
||||
end = endKv.getRow();
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Problem reading orphan file " + hfile + ", skipping");
|
||||
|
|
|
@ -3438,7 +3438,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
|
||||
int i;
|
||||
for (i = 0; i < minLen
|
||||
&& KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
|
||||
&& CellComparator.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
|
||||
++i) {}
|
||||
|
||||
if (additionalMsg == null) {
|
||||
|
|
|
@ -344,7 +344,7 @@ public class HFilePerformanceEvaluation {
|
|||
writer = HFile.getWriterFactoryNoCache(conf)
|
||||
.withPath(fs, mf)
|
||||
.withFileContext(hFileContext)
|
||||
.withComparator(new KeyValue.RawBytesComparator())
|
||||
.withComparator(CellComparator.COMPARATOR)
|
||||
.create();
|
||||
}
|
||||
|
||||
|
|
|
@ -100,7 +100,7 @@ public class MetaMockingUtil {
|
|||
}
|
||||
|
||||
//important: sort the kvs so that binary search work
|
||||
Collections.sort(kvs, KeyValue.META_COMPARATOR);
|
||||
Collections.sort(kvs, CellComparator.META_COMPARATOR);
|
||||
|
||||
return Result.create(kvs);
|
||||
}
|
||||
|
|
|
@ -69,12 +69,12 @@ public class TestSerialization {
|
|||
KeyValue kv = new KeyValue(row, fam, qf, ts, val);
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DataOutputStream dos = new DataOutputStream(baos);
|
||||
long l = KeyValue.write(kv, dos);
|
||||
long l = KeyValueUtil.write(kv, dos);
|
||||
dos.close();
|
||||
byte [] mb = baos.toByteArray();
|
||||
ByteArrayInputStream bais = new ByteArrayInputStream(mb);
|
||||
DataInputStream dis = new DataInputStream(bais);
|
||||
KeyValue deserializedKv = KeyValue.create(dis);
|
||||
KeyValue deserializedKv = KeyValueUtil.create(dis);
|
||||
assertTrue(Bytes.equals(kv.getBuffer(), deserializedKv.getBuffer()));
|
||||
assertEquals(kv.getOffset(), deserializedKv.getOffset());
|
||||
assertEquals(kv.getLength(), deserializedKv.getLength());
|
||||
|
@ -104,7 +104,7 @@ public class TestSerialization {
|
|||
DataInputStream dis = new DataInputStream(bais);
|
||||
|
||||
try {
|
||||
KeyValue.create(dis);
|
||||
KeyValueUtil.create(dis);
|
||||
assertTrue(kv_0.equals(kv_1));
|
||||
} catch (Exception e) {
|
||||
fail("Unexpected Exception" + e.getMessage());
|
||||
|
@ -113,7 +113,7 @@ public class TestSerialization {
|
|||
// length -1
|
||||
try {
|
||||
// even if we have a good kv now in dis we will just pass length with -1 for simplicity
|
||||
KeyValue.create(-1, dis);
|
||||
KeyValueUtil.create(-1, dis);
|
||||
fail("Expected corrupt stream");
|
||||
} catch (Exception e) {
|
||||
assertEquals("Failed read -1 bytes, stream corrupt?", e.getMessage());
|
||||
|
|
|
@ -31,6 +31,7 @@ import junit.framework.TestCase;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -70,7 +71,7 @@ public class TestResult extends TestCase {
|
|||
*/
|
||||
public void testResultAsCellScanner() throws IOException {
|
||||
Cell [] cells = genKVs(row, family, value, 1, 10);
|
||||
Arrays.sort(cells, KeyValue.COMPARATOR);
|
||||
Arrays.sort(cells, CellComparator.COMPARATOR);
|
||||
Result r = Result.create(cells);
|
||||
assertSame(r, cells);
|
||||
// Assert I run over same result multiple times.
|
||||
|
@ -92,7 +93,7 @@ public class TestResult extends TestCase {
|
|||
public void testBasicGetColumn() throws Exception {
|
||||
KeyValue [] kvs = genKVs(row, family, value, 1, 100);
|
||||
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
|
||||
Result r = Result.create(kvs);
|
||||
|
||||
|
@ -114,7 +115,7 @@ public class TestResult extends TestCase {
|
|||
System.arraycopy(kvs1, 0, kvs, 0, kvs1.length);
|
||||
System.arraycopy(kvs2, 0, kvs, kvs1.length, kvs2.length);
|
||||
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
|
||||
Result r = Result.create(kvs);
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
|
@ -131,7 +132,7 @@ public class TestResult extends TestCase {
|
|||
public void testBasicGetValue() throws Exception {
|
||||
KeyValue [] kvs = genKVs(row, family, value, 1, 100);
|
||||
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
|
||||
Result r = Result.create(kvs);
|
||||
|
||||
|
@ -151,7 +152,7 @@ public class TestResult extends TestCase {
|
|||
System.arraycopy(kvs1, 0, kvs, 0, kvs1.length);
|
||||
System.arraycopy(kvs2, 0, kvs, kvs1.length, kvs2.length);
|
||||
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
|
||||
Result r = Result.create(kvs);
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
|
@ -165,7 +166,7 @@ public class TestResult extends TestCase {
|
|||
public void testBasicLoadValue() throws Exception {
|
||||
KeyValue [] kvs = genKVs(row, family, value, 1, 100);
|
||||
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
|
||||
Result r = Result.create(kvs);
|
||||
ByteBuffer loadValueBuffer = ByteBuffer.allocate(1024);
|
||||
|
@ -190,7 +191,7 @@ public class TestResult extends TestCase {
|
|||
System.arraycopy(kvs1, 0, kvs, 0, kvs1.length);
|
||||
System.arraycopy(kvs2, 0, kvs, kvs1.length, kvs2.length);
|
||||
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
|
||||
ByteBuffer loadValueBuffer = ByteBuffer.allocate(1024);
|
||||
|
||||
|
@ -279,7 +280,7 @@ public class TestResult extends TestCase {
|
|||
|
||||
KeyValue [] kvs = genKVs(Bytes.toBytes(rowSB.toString()), family,
|
||||
Bytes.toBytes(valueSB.toString()), 1, n);
|
||||
Arrays.sort(kvs, KeyValue.COMPARATOR);
|
||||
Arrays.sort(kvs, CellComparator.COMPARATOR);
|
||||
ByteBuffer loadValueBuffer = ByteBuffer.allocate(1024);
|
||||
Result r = Result.create(kvs);
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -107,13 +108,13 @@ public class TestCellMessageCodec {
|
|||
Codec.Decoder decoder = cmc.getDecoder(dis);
|
||||
assertTrue(decoder.advance());
|
||||
Cell c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv1));
|
||||
assertTrue(CellUtil.equals(c, kv1));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv2));
|
||||
assertTrue(CellUtil.equals(c, kv2));
|
||||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellComparator.equals(c, kv3));
|
||||
assertTrue(CellUtil.equals(c, kv3));
|
||||
assertFalse(decoder.advance());
|
||||
dis.close();
|
||||
assertEquals(offset, cis.getCount());
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.List;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
|
@ -152,7 +153,7 @@ public class TestDependentColumnFilter {
|
|||
for (boolean done = true; done; i++) {
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
CellComparator.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
if (results.isEmpty()) break;
|
||||
cells += results.size();
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.List;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -1619,7 +1620,7 @@ public class TestFilter {
|
|||
for (boolean done = true; done; i++) {
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
CellComparator.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
if (results.isEmpty()) break;
|
||||
assertTrue("Scanned too many rows! Only expected " + expectedRows +
|
||||
|
@ -1641,7 +1642,7 @@ public class TestFilter {
|
|||
for (boolean done = true; done; i++) {
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
CellComparator.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
if(results.isEmpty()) break;
|
||||
assertTrue("Scanned too many rows! Only expected " + expectedRows +
|
||||
|
@ -1663,7 +1664,7 @@ public class TestFilter {
|
|||
for (boolean done = true; done; row++) {
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
CellComparator.COMPARATOR);
|
||||
if(results.isEmpty()) break;
|
||||
assertTrue("Scanned too many keys! Only expected " + kvs.length +
|
||||
" total but already scanned " + (results.size() + idx) +
|
||||
|
@ -1694,7 +1695,7 @@ public class TestFilter {
|
|||
for (boolean more = true; more; row++) {
|
||||
more = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
CellComparator.COMPARATOR);
|
||||
if(results.isEmpty()) break;
|
||||
assertTrue("Scanned too many keys! Only expected " + kvs.length +
|
||||
" total but already scanned " + (results.size() + idx) +
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -458,7 +459,7 @@ public class TestFilterList {
|
|||
// Should take the min if given two hints
|
||||
FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
|
||||
Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
minKeyValue));
|
||||
|
||||
// Should have no hint if any filter has no hint
|
||||
|
@ -473,7 +474,7 @@ public class TestFilterList {
|
|||
// Should give max hint if its the only one
|
||||
filterList = new FilterList(Operator.MUST_PASS_ONE,
|
||||
Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
maxKeyValue));
|
||||
|
||||
// MUST PASS ALL
|
||||
|
@ -482,13 +483,13 @@ public class TestFilterList {
|
|||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
|
||||
filterList.filterKeyValue(null);
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
minKeyValue));
|
||||
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
|
||||
filterList.filterKeyValue(null);
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
maxKeyValue));
|
||||
|
||||
// Should have first hint even if a filter has no hint
|
||||
|
@ -496,17 +497,17 @@ public class TestFilterList {
|
|||
Arrays.asList(
|
||||
new Filter [] { filterNoHint, filterMinHint, filterMaxHint } ));
|
||||
filterList.filterKeyValue(null);
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
minKeyValue));
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
|
||||
filterList.filterKeyValue(null);
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
maxKeyValue));
|
||||
filterList = new FilterList(Operator.MUST_PASS_ALL,
|
||||
Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
|
||||
filterList.filterKeyValue(null);
|
||||
assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null),
|
||||
minKeyValue));
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
|
@ -67,8 +68,8 @@ public class TestSingleColumnValueExcludeFilter {
|
|||
filter.filterRowCells(kvs);
|
||||
|
||||
assertEquals("resultSize", kvs.size(), 2);
|
||||
assertTrue("leftKV1", KeyValue.COMPARATOR.compare(kvs.get(0), kv) == 0);
|
||||
assertTrue("leftKV2", KeyValue.COMPARATOR.compare(kvs.get(1), kv) == 0);
|
||||
assertTrue("leftKV1", CellComparator.COMPARATOR.compare(kvs.get(0), kv) == 0);
|
||||
assertTrue("leftKV2", CellComparator.COMPARATOR.compare(kvs.get(1), kv) == 0);
|
||||
assertFalse("allRemainingWhenMatch", filter.filterAllRemaining());
|
||||
|
||||
// A 'mismatch' situation
|
||||
|
|
|
@ -31,6 +31,7 @@ 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.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
|
@ -101,7 +102,7 @@ public class TestHalfStoreFileReader {
|
|||
HFile.Reader r = HFile.createReader(fs, p, cacheConf, conf);
|
||||
r.loadFileInfo();
|
||||
byte [] midkey = r.midkey();
|
||||
KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
|
||||
KeyValue midKV = KeyValueUtil.createKeyValueFromKey(midkey);
|
||||
midkey = midKV.getRow();
|
||||
|
||||
//System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
|
||||
|
@ -167,7 +168,7 @@ public class TestHalfStoreFileReader {
|
|||
HFile.Reader r = HFile.createReader(fs, p, cacheConf, conf);
|
||||
r.loadFileInfo();
|
||||
byte[] midkey = r.midkey();
|
||||
KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
|
||||
KeyValue midKV = KeyValueUtil.createKeyValueFromKey(midkey);
|
||||
midkey = midKV.getRow();
|
||||
|
||||
Reference bottom = new Reference(midkey, Reference.Range.bottom);
|
||||
|
@ -176,7 +177,7 @@ public class TestHalfStoreFileReader {
|
|||
// Ugly code to get the item before the midkey
|
||||
KeyValue beforeMidKey = null;
|
||||
for (KeyValue item : items) {
|
||||
if (KeyValue.COMPARATOR.compare(item, midKV) >= 0) {
|
||||
if (CellComparator.COMPARATOR.compare(item, midKV) >= 0) {
|
||||
break;
|
||||
}
|
||||
beforeMidKey = item;
|
||||
|
|
|
@ -17,15 +17,33 @@
|
|||
package org.apache.hadoop.hbase.io.encoding;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
||||
@Category({IOTests.class, MediumTests.class})
|
||||
public class TestBufferedDataBlockEncoder {
|
||||
|
||||
byte[] row1 = Bytes.toBytes("row1");
|
||||
byte[] row2 = Bytes.toBytes("row2");
|
||||
byte[] row_1_0 = Bytes.toBytes("row10");
|
||||
|
||||
byte[] fam1 = Bytes.toBytes("fam1");
|
||||
byte[] fam2 = Bytes.toBytes("fam2");
|
||||
byte[] fam_1_2 = Bytes.toBytes("fam12");
|
||||
|
||||
byte[] qual1 = Bytes.toBytes("qual1");
|
||||
byte[] qual2 = Bytes.toBytes("qual2");
|
||||
|
||||
byte[] val = Bytes.toBytes("val");
|
||||
|
||||
@Test
|
||||
public void testEnsureSpaceForKey() {
|
||||
BufferedDataBlockEncoder.SeekerState state =
|
||||
|
@ -41,4 +59,19 @@ public class TestBufferedDataBlockEncoder {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommonPrefixComparators() {
|
||||
KeyValue kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
KeyValue kv2 = new KeyValue(row1, fam_1_2, qual1, 1l, Type.Maximum);
|
||||
assertTrue((BufferedDataBlockEncoder.compareCommonFamilyPrefix(kv1, kv2, 4) < 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row_1_0, fam_1_2, qual1, 1l, Type.Maximum);
|
||||
assertTrue((BufferedDataBlockEncoder.compareCommonRowPrefix(kv1, kv2, 4) < 0));
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual2, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
|
||||
assertTrue((BufferedDataBlockEncoder.compareCommonQualifierPrefix(kv1, kv2, 4) > 0));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -185,7 +186,7 @@ public class TestDataBlockEncoders {
|
|||
.withIncludesTags(includesTags)
|
||||
.withCompression(Compression.Algorithm.NONE)
|
||||
.build();
|
||||
DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
|
||||
DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(CellComparator.COMPARATOR,
|
||||
encoder.newDataBlockDecodingContext(meta));
|
||||
seeker.setCurrentBuffer(encodedBuffer);
|
||||
encodedSeekers.add(seeker);
|
||||
|
@ -251,7 +252,7 @@ public class TestDataBlockEncoders {
|
|||
.withIncludesTags(includesTags)
|
||||
.withCompression(Compression.Algorithm.NONE)
|
||||
.build();
|
||||
DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
|
||||
DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(CellComparator.COMPARATOR,
|
||||
encoder.newDataBlockDecodingContext(meta));
|
||||
seeker.setCurrentBuffer(encodedBuffer);
|
||||
int i = 0;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue