HBASE-18995 Move methods that are for internal usage from CellUtil to Private util class (Ramkrishna Vasudevan)
This commit is contained in:
parent
f981de5bb9
commit
250a8bc7ad
|
@ -69,7 +69,7 @@ class AllowPartialScanResultCache implements ScanResultCache {
|
|||
if (i == results.length) {
|
||||
return EMPTY_RESULT_ARRAY;
|
||||
}
|
||||
if (lastResultPartial && !CellUtil.matchingRow(lastCell, results[0].getRow())) {
|
||||
if (lastResultPartial && !CellUtil.matchingRows(lastCell, results[0].getRow())) {
|
||||
// there is a row change, so increase numberOfCompleteRows
|
||||
numberOfCompleteRows++;
|
||||
}
|
||||
|
|
|
@ -135,7 +135,7 @@ public class BatchScanResultCache implements ScanResultCache {
|
|||
// there is a row change
|
||||
regroupedResults.add(createCompletedResult());
|
||||
}
|
||||
} else if (lastResultPartial && !CellUtil.matchingRow(lastCell, result.getRow())) {
|
||||
} else if (lastResultPartial && !CellUtil.matchingRows(lastCell, result.getRow())) {
|
||||
// As for batched scan we may return partial results to user if we reach the batch limit, so
|
||||
// here we need to use lastCell to determine if there is row change and increase
|
||||
// numberOfCompleteRows.
|
||||
|
|
|
@ -40,9 +40,9 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -320,7 +320,7 @@ public final class ConnectionUtils {
|
|||
long estimatedHeapSizeOfResult = 0;
|
||||
// We don't make Iterator here
|
||||
for (Cell cell : rs.rawCells()) {
|
||||
estimatedHeapSizeOfResult += CellUtil.estimatedHeapSizeOf(cell);
|
||||
estimatedHeapSizeOfResult += PrivateCellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
return estimatedHeapSizeOfResult;
|
||||
}
|
||||
|
@ -331,7 +331,7 @@ public final class ConnectionUtils {
|
|||
return result;
|
||||
}
|
||||
// not the same row
|
||||
if (!CellUtil.matchingRow(keepCellsAfter, result.getRow(), 0, result.getRow().length)) {
|
||||
if (!PrivateCellUtil.matchingRows(keepCellsAfter, result.getRow(), 0, result.getRow().length)) {
|
||||
return result;
|
||||
}
|
||||
Cell[] rawCells = result.rawCells();
|
||||
|
@ -462,7 +462,7 @@ public final class ConnectionUtils {
|
|||
long resultSize = 0;
|
||||
for (Result rr : rrs) {
|
||||
for (Cell cell : rr.rawCells()) {
|
||||
resultSize += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
resultSize += PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
}
|
||||
}
|
||||
scanMetrics.countOfBytesInResults.addAndGet(resultSize);
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
|
@ -175,7 +174,7 @@ public class Delete extends Mutation implements Comparable<Row> {
|
|||
throw new IOException("The recently added KeyValue is not of type "
|
||||
+ "delete. Rowkey: " + Bytes.toStringBinary(this.row));
|
||||
}
|
||||
if (!CellUtil.matchingRow(kv, this.row)) {
|
||||
if (!CellUtil.matchingRows(kv, this.row)) {
|
||||
throw new WrongRowIOException("The row in " + kv.toString() +
|
||||
" doesn't match the original one " + Bytes.toStringBinary(this.row));
|
||||
}
|
||||
|
|
|
@ -96,7 +96,7 @@ public class Increment extends Mutation implements Comparable<Row> {
|
|||
byte [] family = CellUtil.cloneFamily(cell);
|
||||
List<Cell> list = getCellList(family);
|
||||
//Checking that the row of the kv is the same as the put
|
||||
if (!CellUtil.matchingRow(cell, this.row)) {
|
||||
if (!CellUtil.matchingRows(cell, this.row)) {
|
||||
throw new WrongRowIOException("The row in " + cell +
|
||||
" doesn't match the original one " + Bytes.toStringBinary(this.row));
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.CellScannable;
|
|||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
|
@ -225,7 +226,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
|||
c.getQualifierLength()));
|
||||
stringMap.put("timestamp", c.getTimestamp());
|
||||
stringMap.put("vlen", c.getValueLength());
|
||||
List<Tag> tags = CellUtil.getTags(c);
|
||||
List<Tag> tags = PrivateCellUtil.getTags(c);
|
||||
if (tags != null) {
|
||||
List<String> tagsString = new ArrayList<>(tags.size());
|
||||
for (Tag t : tags) {
|
||||
|
@ -437,7 +438,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
|||
size * ClassSize.REFERENCE);
|
||||
|
||||
for(Cell cell : entry.getValue()) {
|
||||
heapsize += CellUtil.estimatedHeapSizeOf(cell);
|
||||
heapsize += PrivateCellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
}
|
||||
heapsize += getAttributeSize();
|
||||
|
|
|
@ -318,7 +318,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
|
|||
byte [] family = CellUtil.cloneFamily(kv);
|
||||
List<Cell> list = getCellList(family);
|
||||
//Checking that the row of the kv is the same as the put
|
||||
if (!CellUtil.matchingRow(kv, this.row)) {
|
||||
if (!CellUtil.matchingRows(kv, this.row)) {
|
||||
throw new WrongRowIOException("The row in " + kv.toString() +
|
||||
" doesn't match the original one " + Bytes.toStringBinary(this.row));
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.CellScannable;
|
|||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -297,7 +298,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
byte[] familyNotNull = notNullBytes(family);
|
||||
byte[] qualifierNotNull = notNullBytes(qualifier);
|
||||
Cell searchTerm =
|
||||
CellUtil.createFirstOnRow(kvs[0].getRowArray(),
|
||||
PrivateCellUtil.createFirstOnRow(kvs[0].getRowArray(),
|
||||
kvs[0].getRowOffset(), kvs[0].getRowLength(),
|
||||
familyNotNull, 0, (byte)familyNotNull.length,
|
||||
qualifierNotNull, 0, qualifierNotNull.length);
|
||||
|
@ -408,7 +409,8 @@ public class Result implements CellScannable, CellScanner {
|
|||
if (pos == -1) {
|
||||
return null;
|
||||
}
|
||||
if (CellUtil.matchingColumn(kvs[pos], family, foffset, flength, qualifier, qoffset, qlength)) {
|
||||
if (PrivateCellUtil.matchingColumn(kvs[pos], family, foffset, flength, qualifier, qoffset,
|
||||
qlength)) {
|
||||
return kvs[pos];
|
||||
}
|
||||
return null;
|
||||
|
@ -858,7 +860,7 @@ public class Result implements CellScannable, CellScanner {
|
|||
return size;
|
||||
}
|
||||
for (Cell c : result.rawCells()) {
|
||||
size += CellUtil.estimatedHeapSizeOf(c);
|
||||
size += PrivateCellUtil.estimatedHeapSizeOf(c);
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.math.RoundingMode;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
|
@ -49,7 +50,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
|
|||
if (kv == null || CellUtil.cloneValue(kv) == null) {
|
||||
return null;
|
||||
}
|
||||
return CellUtil.getValueAsBigDecimal(kv).setScale(2, RoundingMode.HALF_EVEN);
|
||||
return PrivateCellUtil.getValueAsBigDecimal(kv).setScale(2, RoundingMode.HALF_EVEN);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.client.coprocessor;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
|
@ -48,7 +48,7 @@ public class DoubleColumnInterpreter extends ColumnInterpreter<Double, Double,
|
|||
throws IOException {
|
||||
if (c == null || c.getValueLength() != Bytes.SIZEOF_DOUBLE)
|
||||
return null;
|
||||
return CellUtil.getValueAsDouble(c);
|
||||
return PrivateCellUtil.getValueAsDouble(c);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.client.coprocessor;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
|
@ -47,7 +47,7 @@ public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
|
|||
throws IOException {
|
||||
if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG)
|
||||
return null;
|
||||
return CellUtil.getValueAsLong(kv);
|
||||
return PrivateCellUtil.getValueAsLong(kv);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.ArrayList;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
|
@ -147,7 +148,7 @@ public class ColumnPaginationFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return CellUtil.createFirstOnRowCol(cell, columnOffset, 0, columnOffset.length);
|
||||
return PrivateCellUtil.createFirstOnRowCol(cell, columnOffset, 0, columnOffset.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.ArrayList;
|
|||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
|
@ -152,7 +152,7 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return CellUtil.createFirstOnRowCol(cell, prefix, 0, prefix.length);
|
||||
return PrivateCellUtil.createFirstOnRowCol(cell, prefix, 0, prefix.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.ArrayList;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -224,7 +225,7 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return CellUtil.createFirstOnRowCol(cell, this.minColumn, 0, len(this.minColumn));
|
||||
return PrivateCellUtil.createFirstOnRowCol(cell, this.minColumn, 0, len(this.minColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,8 +23,8 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
|
@ -137,7 +137,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareRow(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareRow(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
|
@ -146,7 +146,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (op == CompareOperator.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareRow(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareRow(cell, comparator);
|
||||
return compare(op, compareResult);
|
||||
}
|
||||
|
||||
|
@ -160,7 +160,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareFamily(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareFamily(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
|
@ -169,7 +169,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (op == CompareOperator.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareFamily(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareFamily(cell, comparator);
|
||||
return compare(op, compareResult);
|
||||
}
|
||||
|
||||
|
@ -184,7 +184,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareQualifier(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareQualifier(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
|
@ -194,7 +194,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (op == CompareOperator.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareQualifier(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareQualifier(cell, comparator);
|
||||
return compare(op, compareResult);
|
||||
}
|
||||
|
||||
|
@ -209,7 +209,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (compareOp == CompareOp.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareValue(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareValue(cell, comparator);
|
||||
return compare(compareOp, compareResult);
|
||||
}
|
||||
|
||||
|
@ -218,7 +218,7 @@ public abstract class CompareFilter extends FilterBase {
|
|||
if (op == CompareOperator.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = CellUtil.compareValue(cell, comparator);
|
||||
int compareResult = PrivateCellUtil.compareValue(cell, comparator);
|
||||
return compare(op, compareResult);
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.PriorityQueue;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -187,7 +188,7 @@ public class FuzzyRowFilter extends FilterBase {
|
|||
return null;
|
||||
}
|
||||
byte[] nextRowKey = tracker.nextRow();
|
||||
return CellUtil.createFirstOnRow(nextRowKey, 0, (short) nextRowKey.length);
|
||||
return PrivateCellUtil.createFirstOnRow(nextRowKey, 0, (short) nextRowKey.length);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.List;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -138,7 +139,7 @@ public class MultiRowRangeFilter extends FilterBase {
|
|||
@Override
|
||||
public Cell getNextCellHint(Cell currentKV) {
|
||||
// skip to the next range's start row
|
||||
return CellUtil.createFirstOnRow(range.startRow, 0,
|
||||
return PrivateCellUtil.createFirstOnRow(range.startRow, 0,
|
||||
(short) range.startRow.length);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
@ -164,7 +165,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
|
|||
|
||||
@Override
|
||||
public Cell getNextCellHint(Cell cell) {
|
||||
return CellUtil.createFirstOnRowCol(cell, hint, 0, hint.length);
|
||||
return PrivateCellUtil.createFirstOnRowCol(cell, hint, 0, hint.length);
|
||||
}
|
||||
|
||||
public TreeSet<byte []> createTreeSet() {
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.ArrayList;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
|
@ -273,7 +274,7 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
}
|
||||
|
||||
private boolean filterColumnValue(final Cell cell) {
|
||||
int compareResult = CellUtil.compareValue(cell, this.comparator);
|
||||
int compareResult = PrivateCellUtil.compareValue(cell, this.comparator);
|
||||
return CompareFilter.compare(this.op, compareResult);
|
||||
}
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.List;
|
|||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
|
@ -150,14 +150,14 @@ public class TimestampsFilter extends FilterBase {
|
|||
// but it's always better to be extra safe and protect against future
|
||||
// behavioral changes.
|
||||
|
||||
return CellUtil.createLastOnRowCol(currentCell);
|
||||
return PrivateCellUtil.createLastOnRowCol(currentCell);
|
||||
}
|
||||
|
||||
// Since we know the nextTimestampObject isn't null here there must still be
|
||||
// timestamps that can be included. Cast the Long to a long and return the
|
||||
// a cell with the current row/cf/col and the next found timestamp.
|
||||
long nextTimestamp = nextTimestampObject;
|
||||
return CellUtil.createFirstOnRowColTS(currentCell, nextTimestamp);
|
||||
return PrivateCellUtil.createFirstOnRowColTS(currentCell, nextTimestamp);
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -53,44 +54,44 @@ public class TestComparators {
|
|||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
ByteArrayComparable comparable = new BinaryComparator(r1);
|
||||
assertEquals(0, CellUtil.compareRow(bbCell, comparable));
|
||||
assertEquals(0, CellUtil.compareRow(kv, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareRow(bbCell, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareRow(kv, comparable));
|
||||
kv = new KeyValue(r0, f, q1, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.compareRow(bbCell, comparable) > 0);
|
||||
assertTrue(CellUtil.compareRow(kv, comparable) > 0);
|
||||
assertTrue(PrivateCellUtil.compareRow(bbCell, comparable) > 0);
|
||||
assertTrue(PrivateCellUtil.compareRow(kv, comparable) > 0);
|
||||
kv = new KeyValue(r2, f, q1, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.compareRow(bbCell, comparable) < 0);
|
||||
assertTrue(CellUtil.compareRow(kv, comparable) < 0);
|
||||
assertTrue(PrivateCellUtil.compareRow(bbCell, comparable) < 0);
|
||||
assertTrue(PrivateCellUtil.compareRow(kv, comparable) < 0);
|
||||
// Qualifier compare
|
||||
comparable = new BinaryPrefixComparator(Bytes.toBytes("qual"));
|
||||
assertEquals(0, CellUtil.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, CellUtil.compareQualifier(kv, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareQualifier(kv, comparable));
|
||||
kv = new KeyValue(r2, f, q2, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellUtil.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, CellUtil.compareQualifier(kv, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareQualifier(kv, comparable));
|
||||
kv = new KeyValue(r2, f, q3, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.compareQualifier(bbCell, comparable) < 0);
|
||||
assertTrue(CellUtil.compareQualifier(kv, comparable) < 0);
|
||||
assertTrue(PrivateCellUtil.compareQualifier(bbCell, comparable) < 0);
|
||||
assertTrue(PrivateCellUtil.compareQualifier(kv, comparable) < 0);
|
||||
// Value compare
|
||||
comparable = new LongComparator(l1);
|
||||
assertEquals(0, CellUtil.compareValue(bbCell, comparable));
|
||||
assertEquals(0, CellUtil.compareValue(kv, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareValue(bbCell, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareValue(kv, comparable));
|
||||
kv = new KeyValue(r1, f, q1, v2);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.compareValue(bbCell, comparable) < 0);
|
||||
assertTrue(CellUtil.compareValue(kv, comparable) < 0);
|
||||
assertTrue(PrivateCellUtil.compareValue(bbCell, comparable) < 0);
|
||||
assertTrue(PrivateCellUtil.compareValue(kv, comparable) < 0);
|
||||
// Family compare
|
||||
comparable = new SubstringComparator("cf");
|
||||
assertEquals(0, CellUtil.compareFamily(bbCell, comparable));
|
||||
assertEquals(0, CellUtil.compareFamily(kv, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareFamily(bbCell, comparable));
|
||||
assertEquals(0, PrivateCellUtil.compareFamily(kv, comparable));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
import org.apache.hadoop.hbase.codec.KeyValueCodec;
|
||||
|
@ -91,7 +92,7 @@ public class TestCellBlockBuilder {
|
|||
static CellScanner getSizedCellScanner(final Cell[] cells) {
|
||||
int size = -1;
|
||||
for (Cell cell : cells) {
|
||||
size += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
size += PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
}
|
||||
final int totalSize = ClassSize.align(size);
|
||||
final CellScanner cellScanner = CellUtil.createCellScanner(cells);
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -24,7 +24,6 @@ import java.io.IOException;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ArrayUtils;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -144,7 +143,7 @@ public class IndividualBytesFieldCell implements ExtendedCell {
|
|||
ByteBufferUtils.putInt(out, getValueLength());
|
||||
|
||||
// Key
|
||||
CellUtil.writeFlatKey(this, out);
|
||||
PrivateCellUtil.writeFlatKey(this, out);
|
||||
|
||||
// Value
|
||||
out.write(getValueArray());
|
||||
|
|
|
@ -1617,7 +1617,7 @@ public class KeyValue implements ExtendedCell {
|
|||
*/
|
||||
@Override
|
||||
public int compare(final Cell left, final Cell right) {
|
||||
return CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.META_COMPARATOR, left, right);
|
||||
return PrivateCellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.META_COMPARATOR, left, right);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1839,7 +1839,7 @@ public class KeyValue implements ExtendedCell {
|
|||
}
|
||||
|
||||
public int compareOnlyKeyPortion(Cell left, Cell right) {
|
||||
return CellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, right);
|
||||
return PrivateCellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, left, right);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -85,7 +85,7 @@ public class KeyValueTestUtil {
|
|||
for (Cell kv1 : kvCollection1) {
|
||||
boolean found = false;
|
||||
for (Cell kv2 : kvCollection2) {
|
||||
if (CellUtil.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
|
||||
if (PrivateCellUtil.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
|
||||
}
|
||||
if (!found) return false;
|
||||
}
|
||||
|
|
|
@ -163,7 +163,7 @@ public class KeyValueUtil {
|
|||
pos = CellUtil.copyValueTo(cell, output, pos);
|
||||
if (withTags && (cell.getTagsLength() > 0)) {
|
||||
pos = Bytes.putAsShort(output, pos, cell.getTagsLength());
|
||||
pos = CellUtil.copyTagTo(cell, output, pos);
|
||||
pos = PrivateCellUtil.copyTagsTo(cell, output, pos);
|
||||
}
|
||||
return pos;
|
||||
}
|
||||
|
@ -179,7 +179,7 @@ public class KeyValueUtil {
|
|||
int tagsLength = cell.getTagsLength();
|
||||
if (withTags && (tagsLength > 0)) {
|
||||
offset = ByteBufferUtils.putAsShort(buf, offset, tagsLength);// Tags length
|
||||
offset = CellUtil.copyTagTo(cell, buf, offset);// Tags bytes
|
||||
offset = PrivateCellUtil.copyTagsTo(cell, buf, offset);// Tags bytes
|
||||
}
|
||||
return offset;
|
||||
}
|
||||
|
|
|
@ -232,7 +232,7 @@ public final class TagUtil {
|
|||
* Add to <code>tagsOrNull</code> any Tags <code>cell</code> is carrying or null if none.
|
||||
*/
|
||||
public static List<Tag> carryForwardTags(final List<Tag> tagsOrNull, final Cell cell) {
|
||||
Iterator<Tag> itr = CellUtil.tagsIterator(cell);
|
||||
Iterator<Tag> itr = PrivateCellUtil.tagsIterator(cell);
|
||||
if (itr == EMPTY_TAGS_ITR) {
|
||||
// If no Tags, return early.
|
||||
return tagsOrNull;
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ExtendedCell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
|
@ -749,7 +750,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
@Override
|
||||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
keyOnlyKV.setKey(current.keyBuffer, 0, current.keyLength);
|
||||
return CellUtil.compareKeyIgnoresMvcc(comparator, key, keyOnlyKV);
|
||||
return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, key, keyOnlyKV);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1027,9 +1028,9 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
// the tags using Dictionary compression in such a case
|
||||
if (tagCompressionContext != null) {
|
||||
// Not passing tagsLength considering that parsing of the tagsLength is not costly
|
||||
CellUtil.compressTags(out, cell, tagCompressionContext);
|
||||
PrivateCellUtil.compressTags(out, cell, tagCompressionContext);
|
||||
} else {
|
||||
CellUtil.writeTags(out, cell, tagsLength);
|
||||
PrivateCellUtil.writeTags(out, cell, tagsLength);
|
||||
}
|
||||
}
|
||||
size += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
|
||||
|
|
|
@ -23,7 +23,7 @@ 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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -229,11 +229,11 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
// put column family
|
||||
byte familyLength = cell.getFamilyLength();
|
||||
out.write(familyLength);
|
||||
CellUtil.writeFamily(out, cell, familyLength);
|
||||
PrivateCellUtil.writeFamily(out, cell, familyLength);
|
||||
} else {
|
||||
// Finding common prefix
|
||||
int preKeyLength = KeyValueUtil.keyLength(prevCell);
|
||||
commonPrefix = CellUtil.findCommonPrefixInFlatKey(cell, prevCell, true, false);
|
||||
commonPrefix = PrivateCellUtil.findCommonPrefixInFlatKey(cell, prevCell, true, false);
|
||||
if (kLength == preKeyLength) {
|
||||
flag |= FLAG_SAME_KEY_LENGTH;
|
||||
}
|
||||
|
@ -281,8 +281,8 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
if (commonPrefix < rLen + KeyValue.ROW_LENGTH_SIZE) {
|
||||
// Previous and current rows are different. Copy the differing part of
|
||||
// the row, skip the column family, and copy the qualifier.
|
||||
CellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out);
|
||||
CellUtil.writeQualifier(out, cell, cell.getQualifierLength());
|
||||
PrivateCellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out);
|
||||
PrivateCellUtil.writeQualifier(out, cell, cell.getQualifierLength());
|
||||
} else {
|
||||
// The common part includes the whole row. As the column family is the
|
||||
// same across the whole file, it will automatically be included in the
|
||||
|
@ -290,7 +290,7 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
// What we write here is the non common part of the qualifier
|
||||
int commonQualPrefix = commonPrefix - (rLen + KeyValue.ROW_LENGTH_SIZE)
|
||||
- (cell.getFamilyLength() + KeyValue.FAMILY_LENGTH_SIZE);
|
||||
CellUtil.writeQualifierSkippingBytes(out, cell, cell.getQualifierLength(),
|
||||
PrivateCellUtil.writeQualifierSkippingBytes(out, cell, cell.getQualifierLength(),
|
||||
commonQualPrefix);
|
||||
}
|
||||
if ((flag & FLAG_TIMESTAMP_IS_DIFF) == 0) {
|
||||
|
@ -302,7 +302,7 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
if ((flag & FLAG_SAME_TYPE) == 0) {
|
||||
out.write(cell.getTypeByte());
|
||||
}
|
||||
CellUtil.writeValue(out, cell, vLength);
|
||||
PrivateCellUtil.writeValue(out, cell, vLength);
|
||||
return kLength + vLength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
|
||||
}
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ 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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -263,14 +263,14 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
ByteBufferUtils.putCompressedInt(out, kLength);
|
||||
ByteBufferUtils.putCompressedInt(out, vLength);
|
||||
ByteBufferUtils.putCompressedInt(out, 0);
|
||||
CellUtil.writeFlatKey(cell, (DataOutput)out);
|
||||
PrivateCellUtil.writeFlatKey(cell, (DataOutput)out);
|
||||
// Write the value part
|
||||
CellUtil.writeValue(out, cell, cell.getValueLength());
|
||||
PrivateCellUtil.writeValue(out, cell, cell.getValueLength());
|
||||
} else {
|
||||
int preKeyLength = KeyValueUtil.keyLength(prevCell);
|
||||
int preValLength = prevCell.getValueLength();
|
||||
// find a common prefix and skip it
|
||||
int commonPrefix = CellUtil.findCommonPrefixInFlatKey(cell, prevCell, true, false);
|
||||
int commonPrefix = PrivateCellUtil.findCommonPrefixInFlatKey(cell, prevCell, true, false);
|
||||
|
||||
if (kLength == preKeyLength) {
|
||||
flag |= FLAG_SAME_KEY_LENGTH;
|
||||
|
@ -291,7 +291,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
// Check if current and previous values are the same. Compare value
|
||||
// length first as an optimization.
|
||||
if (vLength == preValLength
|
||||
&& CellUtil.matchingValue(cell, prevCell, vLength, preValLength)) {
|
||||
&& PrivateCellUtil.matchingValue(cell, prevCell, vLength, preValLength)) {
|
||||
flag |= FLAG_SAME_VALUE;
|
||||
}
|
||||
|
||||
|
@ -307,8 +307,8 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
if (commonPrefix < rLen + KeyValue.ROW_LENGTH_SIZE) {
|
||||
// Previous and current rows are different. Copy the differing part of
|
||||
// the row, skip the column family, and copy the qualifier.
|
||||
CellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out);
|
||||
CellUtil.writeQualifier(out, cell, cell.getQualifierLength());
|
||||
PrivateCellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out);
|
||||
PrivateCellUtil.writeQualifier(out, cell, cell.getQualifierLength());
|
||||
} else {
|
||||
// The common part includes the whole row. As the column family is the
|
||||
// same across the whole file, it will automatically be included in the
|
||||
|
@ -316,7 +316,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
// What we write here is the non common part of the qualifier
|
||||
int commonQualPrefix = commonPrefix - (rLen + KeyValue.ROW_LENGTH_SIZE)
|
||||
- (cell.getFamilyLength() + KeyValue.FAMILY_LENGTH_SIZE);
|
||||
CellUtil.writeQualifierSkippingBytes(out, cell, cell.getQualifierLength(),
|
||||
PrivateCellUtil.writeQualifierSkippingBytes(out, cell, cell.getQualifierLength(),
|
||||
commonQualPrefix);
|
||||
}
|
||||
// Write non common ts part
|
||||
|
@ -329,7 +329,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
|
||||
// Write the value if it is not the same as before.
|
||||
if ((flag & FLAG_SAME_VALUE) == 0) {
|
||||
CellUtil.writeValue(out, cell, vLength);
|
||||
PrivateCellUtil.writeValue(out, cell, vLength);
|
||||
}
|
||||
}
|
||||
return kLength + vLength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.DataOutputStream;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -50,7 +50,7 @@ public class NoneEncoder {
|
|||
int tagsLength = cell.getTagsLength();
|
||||
out.writeShort(tagsLength);
|
||||
if (tagsLength > 0) {
|
||||
CellUtil.writeTags(out, cell, tagsLength);
|
||||
PrivateCellUtil.writeTags(out, cell, tagsLength);
|
||||
}
|
||||
size += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ 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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -60,17 +60,17 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
ByteBufferUtils.putCompressedInt(out, klength);
|
||||
ByteBufferUtils.putCompressedInt(out, vlength);
|
||||
ByteBufferUtils.putCompressedInt(out, 0);
|
||||
CellUtil.writeFlatKey(cell, (DataOutput)out);
|
||||
PrivateCellUtil.writeFlatKey(cell, (DataOutput)out);
|
||||
} else {
|
||||
// find a common prefix and skip it
|
||||
int common = CellUtil.findCommonPrefixInFlatKey(cell, state.prevCell, true, true);
|
||||
int common = PrivateCellUtil.findCommonPrefixInFlatKey(cell, state.prevCell, true, true);
|
||||
ByteBufferUtils.putCompressedInt(out, klength - common);
|
||||
ByteBufferUtils.putCompressedInt(out, vlength);
|
||||
ByteBufferUtils.putCompressedInt(out, common);
|
||||
writeKeyExcludingCommon(cell, common, out);
|
||||
}
|
||||
// Write the value part
|
||||
CellUtil.writeValue(out, cell, vlength);
|
||||
PrivateCellUtil.writeValue(out, cell, vlength);
|
||||
int size = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
|
||||
size += afterEncodingKeyValue(cell, out, encodingContext);
|
||||
state.prevCell = cell;
|
||||
|
@ -83,11 +83,11 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
if (commonPrefix < rLen + KeyValue.ROW_LENGTH_SIZE) {
|
||||
// Previous and current rows are different. Need to write the differing part followed by
|
||||
// cf,q,ts and type
|
||||
CellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out);
|
||||
PrivateCellUtil.writeRowKeyExcludingCommon(cell, rLen, commonPrefix, out);
|
||||
byte fLen = cell.getFamilyLength();
|
||||
out.writeByte(fLen);
|
||||
CellUtil.writeFamily(out, cell, fLen);
|
||||
CellUtil.writeQualifier(out, cell, cell.getQualifierLength());
|
||||
PrivateCellUtil.writeFamily(out, cell, fLen);
|
||||
PrivateCellUtil.writeQualifier(out, cell, cell.getQualifierLength());
|
||||
out.writeLong(cell.getTimestamp());
|
||||
out.writeByte(cell.getTypeByte());
|
||||
} else {
|
||||
|
@ -99,7 +99,7 @@ public class PrefixKeyDeltaEncoder extends BufferedDataBlockEncoder {
|
|||
int commonQualPrefix = Math.min(commonPrefix, qLen);
|
||||
int qualPartLenToWrite = qLen - commonQualPrefix;
|
||||
if (qualPartLenToWrite > 0) {
|
||||
CellUtil.writeQualifierSkippingBytes(out, cell, qLen, commonQualPrefix);
|
||||
PrivateCellUtil.writeQualifierSkippingBytes(out, cell, qLen, commonQualPrefix);
|
||||
}
|
||||
commonPrefix -= commonQualPrefix;
|
||||
// Common part in TS also?
|
||||
|
|
|
@ -24,6 +24,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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedKeyValue;
|
||||
|
@ -190,7 +191,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
}
|
||||
do {
|
||||
int comp;
|
||||
comp = CellUtil.compareKeyIgnoresMvcc(comparator, seekCell, current.currentKey);
|
||||
comp = PrivateCellUtil.compareKeyIgnoresMvcc(comparator, seekCell, current.currentKey);
|
||||
if (comp == 0) { // exact match
|
||||
if (seekBefore) {
|
||||
if (!previous.isValid()) {
|
||||
|
@ -244,7 +245,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
|
||||
@Override
|
||||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
return CellUtil.compareKeyIgnoresMvcc(comparator, key, current.currentKey);
|
||||
return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, key, current.currentKey);
|
||||
}
|
||||
|
||||
protected void decodeFirst() {
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
|
@ -30,7 +31,7 @@ public class RowBloomHashKey extends CellHashKey {
|
|||
|
||||
@Override
|
||||
public byte get(int offset) {
|
||||
return CellUtil.getRowByte(t, offset);
|
||||
return PrivateCellUtil.getRowByte(t, offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -57,7 +58,7 @@ public class RowColBloomHashKey extends CellHashKey {
|
|||
}
|
||||
int refLen = Bytes.SIZEOF_SHORT + rowLength;
|
||||
if (offset < refLen) {
|
||||
return CellUtil.getRowByte(t, offset - Bytes.SIZEOF_SHORT);
|
||||
return PrivateCellUtil.getRowByte(t, offset - Bytes.SIZEOF_SHORT);
|
||||
}
|
||||
if (offset == refLen) {
|
||||
// The fam length should return 0 assuming there is no column family.
|
||||
|
@ -67,7 +68,7 @@ public class RowColBloomHashKey extends CellHashKey {
|
|||
refLen += qualLength + Bytes.SIZEOF_BYTE;
|
||||
// skip the family len because actual cells may have family also
|
||||
if (offset < refLen) {
|
||||
return CellUtil.getQualifierByte(t,
|
||||
return PrivateCellUtil.getQualifierByte(t,
|
||||
offset - (Bytes.SIZEOF_SHORT + rowLength + Bytes.SIZEOF_BYTE));
|
||||
}
|
||||
// TODO : check if ts and type can be removed
|
||||
|
|
|
@ -166,9 +166,9 @@ public class TestByteBufferKeyValue {
|
|||
Tag tag2 = resTags.get(1);
|
||||
assertEquals(tag2.getType(), tag2.getType());
|
||||
assertEquals(TagUtil.getValueAsString(t2), TagUtil.getValueAsString(tag2));
|
||||
Tag res = CellUtil.getTag(offheapKV, (byte) 2);
|
||||
Tag res = PrivateCellUtil.getTag(offheapKV, (byte) 2);
|
||||
assertEquals(TagUtil.getValueAsString(t2), TagUtil.getValueAsString(tag2));
|
||||
res = CellUtil.getTag(offheapKV, (byte) 3);
|
||||
res = PrivateCellUtil.getTag(offheapKV, (byte) 3);
|
||||
assertNull(res);
|
||||
}
|
||||
|
||||
|
|
|
@ -72,23 +72,28 @@ public class TestCellComparator {
|
|||
public void testCompareCellWithKey() throws Exception {
|
||||
KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
|
||||
KeyValue kv2 = new KeyValue(row2, fam1, qual1, val);
|
||||
assertTrue((CellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) < 0);
|
||||
assertTrue(
|
||||
(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) < 0);
|
||||
|
||||
kv1 = new KeyValue(row1, fam2, qual1, val);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, val);
|
||||
assertTrue((CellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
assertTrue(
|
||||
(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, val);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 2l, val);
|
||||
assertTrue((CellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
assertTrue(
|
||||
(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
|
||||
assertTrue((CellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
assertTrue(
|
||||
(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) > 0);
|
||||
|
||||
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
assertTrue((CellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) == 0);
|
||||
assertTrue(
|
||||
(PrivateCellUtil.compare(comparator, kv1, kv2.getKey(), 0, kv2.getKey().length)) == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -23,7 +23,6 @@ import static org.junit.Assert.assertFalse;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -266,43 +265,43 @@ public class TestCellUtil {
|
|||
byte[] d = Bytes.toBytes("d");
|
||||
|
||||
// overlaps
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, b, a, b));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, c, a, b));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, b, a, c));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(b, c, a, c));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, c, b, c));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, d, b, c));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(b, c, a, d));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, b, a, b));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, c, a, b));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, b, a, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(b, c, a, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, c, b, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, d, b, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(b, c, a, d));
|
||||
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(empty, b, a, b));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(empty, b, a, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(empty, b, a, b));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(empty, b, a, c));
|
||||
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, b, empty, b));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, b, empty, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, b, empty, b));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, b, empty, c));
|
||||
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, empty, a, b));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, empty, a, c));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, empty, a, b));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, empty, a, c));
|
||||
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(a, b, empty, empty));
|
||||
Assert.assertTrue(CellUtil.overlappingKeys(empty, empty, a, b));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(a, b, empty, empty));
|
||||
Assert.assertTrue(PrivateCellUtil.overlappingKeys(empty, empty, a, b));
|
||||
|
||||
// non overlaps
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(a, b, c, d));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(c, d, a, b));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(a, b, c, d));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(c, d, a, b));
|
||||
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(b, c, c, d));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(b, c, c, empty));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(b, c, d, empty));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(c, d, b, c));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(c, empty, b, c));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(d, empty, b, c));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(b, c, c, d));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(b, c, c, empty));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(b, c, d, empty));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(c, d, b, c));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(c, empty, b, c));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(d, empty, b, c));
|
||||
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(b, c, a, b));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(b, c, empty, b));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(b, c, empty, a));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(a,b, b, c));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(empty, b, b, c));
|
||||
Assert.assertFalse(CellUtil.overlappingKeys(empty, a, b, c));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(b, c, a, b));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(b, c, empty, b));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(b, c, empty, a));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(a,b, b, c));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(empty, b, b, c));
|
||||
Assert.assertFalse(PrivateCellUtil.overlappingKeys(empty, a, b, c));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -310,43 +309,43 @@ public class TestCellUtil {
|
|||
// The whole key matching case
|
||||
KeyValue kv1 = new KeyValue("r1".getBytes(), "f1".getBytes(), "q1".getBytes(), null);
|
||||
Assert.assertEquals(kv1.getKeyLength(),
|
||||
CellUtil.findCommonPrefixInFlatKey(kv1, kv1, true, true));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv1, kv1, true, true));
|
||||
Assert.assertEquals(kv1.getKeyLength(),
|
||||
CellUtil.findCommonPrefixInFlatKey(kv1, kv1, false, true));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv1, kv1, false, true));
|
||||
Assert.assertEquals(kv1.getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE,
|
||||
CellUtil.findCommonPrefixInFlatKey(kv1, kv1, true, false));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv1, kv1, true, false));
|
||||
// The rk length itself mismatch
|
||||
KeyValue kv2 = new KeyValue("r12".getBytes(), "f1".getBytes(), "q1".getBytes(), null);
|
||||
Assert.assertEquals(1, CellUtil.findCommonPrefixInFlatKey(kv1, kv2, true, true));
|
||||
Assert.assertEquals(1, PrivateCellUtil.findCommonPrefixInFlatKey(kv1, kv2, true, true));
|
||||
// part of rk is same
|
||||
KeyValue kv3 = new KeyValue("r14".getBytes(), "f1".getBytes(), "q1".getBytes(), null);
|
||||
Assert.assertEquals(KeyValue.ROW_LENGTH_SIZE + "r1".getBytes().length,
|
||||
CellUtil.findCommonPrefixInFlatKey(kv2, kv3, true, true));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv2, kv3, true, true));
|
||||
// entire rk is same but different cf name
|
||||
KeyValue kv4 = new KeyValue("r14".getBytes(), "f2".getBytes(), "q1".getBytes(), null);
|
||||
Assert.assertEquals(KeyValue.ROW_LENGTH_SIZE + kv3.getRowLength() + KeyValue.FAMILY_LENGTH_SIZE
|
||||
+ "f".getBytes().length, CellUtil.findCommonPrefixInFlatKey(kv3, kv4, false, true));
|
||||
+ "f".getBytes().length, PrivateCellUtil.findCommonPrefixInFlatKey(kv3, kv4, false, true));
|
||||
// rk and family are same and part of qualifier
|
||||
KeyValue kv5 = new KeyValue("r14".getBytes(), "f2".getBytes(), "q123".getBytes(), null);
|
||||
Assert.assertEquals(KeyValue.ROW_LENGTH_SIZE + kv3.getRowLength() + KeyValue.FAMILY_LENGTH_SIZE
|
||||
+ kv4.getFamilyLength() + kv4.getQualifierLength(),
|
||||
CellUtil.findCommonPrefixInFlatKey(kv4, kv5, true, true));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv4, kv5, true, true));
|
||||
// rk, cf and q are same. ts differs
|
||||
KeyValue kv6 = new KeyValue("rk".getBytes(), 1234L);
|
||||
KeyValue kv7 = new KeyValue("rk".getBytes(), 1235L);
|
||||
// only last byte out of 8 ts bytes in ts part differs
|
||||
Assert.assertEquals(KeyValue.ROW_LENGTH_SIZE + kv6.getRowLength() + KeyValue.FAMILY_LENGTH_SIZE
|
||||
+ kv6.getFamilyLength() + kv6.getQualifierLength() + 7,
|
||||
CellUtil.findCommonPrefixInFlatKey(kv6, kv7, true, true));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv6, kv7, true, true));
|
||||
// rk, cf, q and ts are same. Only type differs
|
||||
KeyValue kv8 = new KeyValue("rk".getBytes(), 1234L, Type.Delete);
|
||||
Assert.assertEquals(KeyValue.ROW_LENGTH_SIZE + kv6.getRowLength() + KeyValue.FAMILY_LENGTH_SIZE
|
||||
+ kv6.getFamilyLength() + kv6.getQualifierLength() + KeyValue.TIMESTAMP_SIZE,
|
||||
CellUtil.findCommonPrefixInFlatKey(kv6, kv8, true, true));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv6, kv8, true, true));
|
||||
// With out TS_TYPE check
|
||||
Assert.assertEquals(KeyValue.ROW_LENGTH_SIZE + kv6.getRowLength() + KeyValue.FAMILY_LENGTH_SIZE
|
||||
+ kv6.getFamilyLength() + kv6.getQualifierLength(),
|
||||
CellUtil.findCommonPrefixInFlatKey(kv6, kv8, true, false));
|
||||
PrivateCellUtil.findCommonPrefixInFlatKey(kv6, kv8, true, false));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -427,7 +426,7 @@ public class TestCellUtil {
|
|||
byte[] vDest = CellUtil.cloneValue(bbCell);
|
||||
assertTrue(Bytes.equals(v, vDest));
|
||||
byte[] tDest = new byte[tags.length];
|
||||
CellUtil.copyTagTo(bbCell, tDest, 0);
|
||||
PrivateCellUtil.copyTagsTo(bbCell, tDest, 0);
|
||||
assertTrue(Bytes.equals(tags, tDest));
|
||||
}
|
||||
|
||||
|
@ -447,7 +446,7 @@ public class TestCellUtil {
|
|||
Cell bbCell2 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingRows(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingRow(bbCell1, r));
|
||||
assertTrue(CellUtil.matchingRows(bbCell1, r));
|
||||
assertTrue(CellUtil.matchingFamily(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingFamily(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingFamily(bbCell1, f));
|
||||
|
@ -475,20 +474,20 @@ public class TestCellUtil {
|
|||
KeyValue kv = new KeyValue(r, f, q, v);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertEquals(ri, CellUtil.getRowAsInt(bbCell));
|
||||
assertEquals(vl, CellUtil.getValueAsLong(bbCell));
|
||||
assertEquals(ri, PrivateCellUtil.getRowAsInt(bbCell));
|
||||
assertEquals(vl, PrivateCellUtil.getValueAsLong(bbCell));
|
||||
double vd = 3005.5;
|
||||
v = Bytes.toBytes(vd);
|
||||
kv = new KeyValue(r, f, q, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertEquals(vd, CellUtil.getValueAsDouble(bbCell), 0.0);
|
||||
assertEquals(vd, PrivateCellUtil.getValueAsDouble(bbCell), 0.0);
|
||||
BigDecimal bd = new BigDecimal(9999);
|
||||
v = Bytes.toBytes(bd);
|
||||
kv = new KeyValue(r, f, q, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
|
||||
assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
|
||||
assertEquals(bd, PrivateCellUtil.getValueAsBigDecimal(bbCell));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -502,7 +501,7 @@ public class TestCellUtil {
|
|||
KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
NonExtendedCell nonExtCell = new NonExtendedCell(kv);
|
||||
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||
int writeCell = CellUtil.writeCell(nonExtCell, os, true);
|
||||
int writeCell = PrivateCellUtil.writeCell(nonExtCell, os, true);
|
||||
byte[] byteArray = os.toByteArray();
|
||||
KeyValue res = new KeyValue(byteArray);
|
||||
assertTrue(CellUtil.equals(kv, res));
|
||||
|
|
|
@ -470,7 +470,7 @@ public class TestKeyValue extends TestCase {
|
|||
}
|
||||
assertTrue(meta1Ok);
|
||||
assertTrue(meta2Ok);
|
||||
Iterator<Tag> tagItr = CellUtil.tagsIterator(kv);
|
||||
Iterator<Tag> tagItr = PrivateCellUtil.tagsIterator(kv);
|
||||
//Iterator<Tag> tagItr = kv.tagsIterator();
|
||||
assertTrue(tagItr.hasNext());
|
||||
Tag next = tagItr.next();
|
||||
|
@ -484,7 +484,7 @@ public class TestKeyValue extends TestCase {
|
|||
Bytes.equals(TagUtil.cloneValue(next), metaValue2);
|
||||
assertFalse(tagItr.hasNext());
|
||||
|
||||
tagItr = CellUtil.tagsIterator(kv);
|
||||
tagItr = PrivateCellUtil.tagsIterator(kv);
|
||||
assertTrue(tagItr.hasNext());
|
||||
next = tagItr.next();
|
||||
assertEquals(10, next.getValueLength());
|
||||
|
|
|
@ -24,8 +24,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ExtendedCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -89,8 +88,8 @@ public class CellSerialization implements Serialization<Cell> {
|
|||
|
||||
@Override
|
||||
public void serialize(Cell kv) throws IOException {
|
||||
dos.writeInt(CellUtil.estimatedSerializedSizeOf(kv) - Bytes.SIZEOF_INT);
|
||||
CellUtil.writeCell(kv, dos, true);
|
||||
dos.writeInt(PrivateCellUtil.estimatedSerializedSizeOf(kv) - Bytes.SIZEOF_INT);
|
||||
PrivateCellUtil.writeCell(kv, dos, true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.mapreduce.Reducer;
|
||||
|
@ -45,7 +45,7 @@ public class CellSortReducer
|
|||
TreeSet<Cell> map = new TreeSet<>(CellComparatorImpl.COMPARATOR);
|
||||
for (Cell kv : kvs) {
|
||||
try {
|
||||
map.add(CellUtil.deepClone(kv));
|
||||
map.add(PrivateCellUtil.deepClone(kv));
|
||||
} catch (CloneNotSupportedException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
|
|
@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
|
@ -247,7 +247,7 @@ public class HFileOutputFormat2
|
|||
}
|
||||
|
||||
byte[] rowKey = CellUtil.cloneRow(kv);
|
||||
int length = (CellUtil.estimatedSerializedSizeOf(kv)) - Bytes.SIZEOF_INT;
|
||||
int length = (PrivateCellUtil.estimatedSerializedSizeOf(kv)) - Bytes.SIZEOF_INT;
|
||||
byte[] family = CellUtil.cloneFamily(kv);
|
||||
byte[] tableNameBytes = null;
|
||||
if (writeMultipleTables) {
|
||||
|
@ -337,7 +337,7 @@ public class HFileOutputFormat2
|
|||
|
||||
// we now have the proper WAL writer. full steam ahead
|
||||
// TODO : Currently in SettableTimeStamp but this will also move to ExtendedCell
|
||||
CellUtil.updateLatestStamp(cell, this.now);
|
||||
PrivateCellUtil.updateLatestStamp(cell, this.now);
|
||||
wl.writer.append(kv);
|
||||
wl.written += length;
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -206,9 +207,9 @@ public class Import extends Configured implements Tool {
|
|||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(CellUtil.estimatedSerializedSizeOfKey(kv));
|
||||
out.writeInt(PrivateCellUtil.estimatedSerializedSizeOfKey(kv));
|
||||
out.writeInt(0);
|
||||
CellUtil.writeFlatKey(kv, out);
|
||||
PrivateCellUtil.writeFlatKey(kv, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -304,7 +305,7 @@ public class Import extends Configured implements Tool {
|
|||
"Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
|
||||
}
|
||||
if (filter == null || !filter.filterRowKey(
|
||||
CellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
|
||||
PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
|
||||
for (Cell kv : value.rawCells()) {
|
||||
kv = filterKv(filter, kv);
|
||||
// skip if we filtered it out
|
||||
|
@ -369,7 +370,7 @@ public class Import extends Configured implements Tool {
|
|||
"Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
|
||||
}
|
||||
if (filter == null || !filter.filterRowKey(
|
||||
CellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
|
||||
PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))) {
|
||||
for (Cell kv : value.rawCells()) {
|
||||
kv = filterKv(filter, kv);
|
||||
// skip if we filtered it out
|
||||
|
@ -412,7 +413,7 @@ public class Import extends Configured implements Tool {
|
|||
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
|
||||
}
|
||||
if (filter == null
|
||||
|| !filter.filterRowKey(CellUtil.createFirstOnRow(row.get(), row.getOffset(),
|
||||
|| !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(),
|
||||
(short) row.getLength()))) {
|
||||
for (Cell kv : value.rawCells()) {
|
||||
kv = filterKv(filter, kv);
|
||||
|
@ -477,7 +478,7 @@ public class Import extends Configured implements Tool {
|
|||
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
|
||||
}
|
||||
if (filter == null
|
||||
|| !filter.filterRowKey(CellUtil.createFirstOnRow(row.get(), row.getOffset(),
|
||||
|| !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(),
|
||||
(short) row.getLength()))) {
|
||||
for (Cell kv : value.rawCells()) {
|
||||
kv = filterKv(filter, kv);
|
||||
|
@ -533,7 +534,7 @@ public class Import extends Configured implements Tool {
|
|||
+ Bytes.toString(key.get(), key.getOffset(), key.getLength()));
|
||||
}
|
||||
if (filter == null
|
||||
|| !filter.filterRowKey(CellUtil.createFirstOnRow(key.get(), key.getOffset(),
|
||||
|| !filter.filterRowKey(PrivateCellUtil.createFirstOnRow(key.get(), key.getOffset(),
|
||||
(short) key.getLength()))) {
|
||||
processKV(key, result, context, put, delete);
|
||||
}
|
||||
|
@ -555,7 +556,7 @@ public class Import extends Configured implements Tool {
|
|||
* submit multiple DeleteFamily tombstones in single Delete request then we are maintaining
|
||||
* only newest in hbase table and ignoring other. Check - HBASE-12065
|
||||
*/
|
||||
if (CellUtil.isDeleteFamily(kv)) {
|
||||
if (PrivateCellUtil.isDeleteFamily(kv)) {
|
||||
Delete deleteFamily = new Delete(key.get());
|
||||
deleteFamily.add(kv);
|
||||
if (durability != null) {
|
||||
|
|
|
@ -25,10 +25,10 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution.HostAndWeight;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
|
||||
import org.apache.hadoop.hbase.client.IsolationLevel;
|
||||
|
@ -363,7 +363,7 @@ public class TableSnapshotInputFormatImpl {
|
|||
if (numSplits > 1) {
|
||||
byte[][] sp = sa.split(hri.getStartKey(), hri.getEndKey(), numSplits, true);
|
||||
for (int i = 0; i < sp.length - 1; i++) {
|
||||
if (CellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(), sp[i],
|
||||
if (PrivateCellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(), sp[i],
|
||||
sp[i + 1])) {
|
||||
// compute HDFS locations from snapshot files (which will get the locations for
|
||||
// referred hfiles)
|
||||
|
@ -379,8 +379,8 @@ public class TableSnapshotInputFormatImpl {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
if (CellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(), hri.getStartKey(),
|
||||
hri.getEndKey())) {
|
||||
if (PrivateCellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(),
|
||||
hri.getStartKey(), hri.getEndKey())) {
|
||||
// compute HDFS locations from snapshot files (which will get the locations for
|
||||
// referred hfiles)
|
||||
List<String> hosts = getBestLocations(conf,
|
||||
|
|
|
@ -211,7 +211,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
// Aggregate as much as possible into a single Put/Delete
|
||||
// operation before writing to the context.
|
||||
if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
|
||||
|| !CellUtil.matchingRow(lastCell, cell)) {
|
||||
|| !CellUtil.matchingRows(lastCell, cell)) {
|
||||
// row or type changed, write out aggregate KVs.
|
||||
if (put != null) {
|
||||
context.write(tableOut, put);
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.ByteBufferCell;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ExtendedCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -226,43 +227,43 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public void setSequenceId(long seqId) throws IOException {
|
||||
CellUtil.setSequenceId(cell, seqId);
|
||||
PrivateCellUtil.setSequenceId(cell, seqId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimestamp(long ts) throws IOException {
|
||||
CellUtil.setTimestamp(cell, ts);
|
||||
PrivateCellUtil.setTimestamp(cell, ts);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimestamp(byte[] ts, int tsOffset) throws IOException {
|
||||
CellUtil.setTimestamp(cell, ts, tsOffset);
|
||||
PrivateCellUtil.setTimestamp(cell, ts, tsOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSize() {
|
||||
return CellUtil.estimatedHeapSizeOf(cell);
|
||||
return PrivateCellUtil.estimatedHeapSizeOf(cell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int write(OutputStream out, boolean withTags) throws IOException {
|
||||
return CellUtil.writeCell(cell, out, withTags);
|
||||
return PrivateCellUtil.writeCell(cell, out, withTags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSerializedSize(boolean withTags) {
|
||||
return CellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
|
||||
return PrivateCellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(ByteBuffer buf, int offset) {
|
||||
CellUtil.writeCellToBuffer(cell, buf, offset);
|
||||
PrivateCellUtil.writeCellToBuffer(cell, buf, offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExtendedCell deepClone() {
|
||||
try {
|
||||
return (ExtendedCell) CellUtil.deepClone(cell);
|
||||
return (ExtendedCell) PrivateCellUtil.deepClone(cell);
|
||||
} catch (CloneNotSupportedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
|
|
@ -714,7 +714,7 @@ public class TestCellBasedHFileOutputFormat2 {
|
|||
assertEquals(FAMILIES.length, res.rawCells().length);
|
||||
Cell first = res.rawCells()[0];
|
||||
for (Cell kv : res.rawCells()) {
|
||||
assertTrue(CellUtil.matchingRow(first, kv));
|
||||
assertTrue(CellUtil.matchingRows(first, kv));
|
||||
assertTrue(Bytes.equals(CellUtil.cloneValue(first), CellUtil.cloneValue(kv)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -375,7 +376,7 @@ public class TestCellBasedImportExport2 {
|
|||
ResultScanner scanner = t.getScanner(s);
|
||||
Result r = scanner.next();
|
||||
Cell[] res = r.rawCells();
|
||||
assertTrue(CellUtil.isDeleteFamily(res[0]));
|
||||
assertTrue(PrivateCellUtil.isDeleteFamily(res[0]));
|
||||
assertEquals(now+4, res[1].getTimestamp());
|
||||
assertEquals(now+3, res[2].getTimestamp());
|
||||
assertTrue(CellUtil.isDelete(res[3]));
|
||||
|
|
|
@ -714,7 +714,7 @@ public class TestHFileOutputFormat2 {
|
|||
assertEquals(FAMILIES.length, res.rawCells().length);
|
||||
Cell first = res.rawCells()[0];
|
||||
for (Cell kv : res.rawCells()) {
|
||||
assertTrue(CellUtil.matchingRow(first, kv));
|
||||
assertTrue(CellUtil.matchingRows(first, kv));
|
||||
assertTrue(Bytes.equals(CellUtil.cloneValue(first), CellUtil.cloneValue(kv)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -375,7 +376,7 @@ public class TestImportExport {
|
|||
ResultScanner scanner = t.getScanner(s);
|
||||
Result r = scanner.next();
|
||||
Cell[] res = r.rawCells();
|
||||
assertTrue(CellUtil.isDeleteFamily(res[0]));
|
||||
assertTrue(PrivateCellUtil.isDeleteFamily(res[0]));
|
||||
assertEquals(now+4, res[1].getTimestamp());
|
||||
assertEquals(now+3, res[2].getTimestamp());
|
||||
assertTrue(CellUtil.isDelete(res[3]));
|
||||
|
|
|
@ -218,8 +218,8 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
|
|||
LOG.debug("Getting results " + res.size());
|
||||
assertTrue(res.size() == 2);
|
||||
List<Cell> kvs = res.listCells();
|
||||
assertTrue(CellUtil.matchingRow(kvs.get(0), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRow(kvs.get(1), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRows(kvs.get(0), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRows(kvs.get(1), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingValue(kvs.get(0), Bytes.toBytes("VALUE" + valueMultiplier)));
|
||||
assertTrue(CellUtil.matchingValue(kvs.get(1),
|
||||
Bytes.toBytes("VALUE" + 2 * valueMultiplier)));
|
||||
|
|
|
@ -448,8 +448,8 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
|
|||
LOG.debug("Getting results " + res.size());
|
||||
assertTrue(res.size() == 2);
|
||||
List<Cell> kvs = res.listCells();
|
||||
assertTrue(CellUtil.matchingRow(kvs.get(0), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRow(kvs.get(1), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRows(kvs.get(0), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRows(kvs.get(1), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingValue(kvs.get(0), Bytes.toBytes("VALUE" + valueMultiplier)));
|
||||
assertTrue(CellUtil.matchingValue(kvs.get(1),
|
||||
Bytes.toBytes("VALUE" + 2 * valueMultiplier)));
|
||||
|
|
|
@ -480,8 +480,8 @@ public class TestImportTsv implements Configurable {
|
|||
numRows++;
|
||||
assertEquals(2, res.size());
|
||||
List<Cell> kvs = res.listCells();
|
||||
assertTrue(CellUtil.matchingRow(kvs.get(0), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRow(kvs.get(1), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRows(kvs.get(0), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingRows(kvs.get(1), Bytes.toBytes("KEY")));
|
||||
assertTrue(CellUtil.matchingValue(kvs.get(0), Bytes.toBytes("VALUE" + valueMultiplier)));
|
||||
assertTrue(CellUtil.matchingValue(kvs.get(1), Bytes.toBytes("VALUE" + 2 * valueMultiplier)));
|
||||
// Only one result set is expected, so let it loop.
|
||||
|
|
|
@ -145,7 +145,7 @@ public class TestSyncTable {
|
|||
Cell sourceCell = sourceCells[j];
|
||||
Cell targetCell = targetCells[j];
|
||||
try {
|
||||
if (!CellUtil.matchingRow(sourceCell, targetCell)) {
|
||||
if (!CellUtil.matchingRows(sourceCell, targetCell)) {
|
||||
Assert.fail("Rows don't match");
|
||||
}
|
||||
if (!CellUtil.matchingFamily(sourceCell, targetCell)) {
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.ByteBufferCell;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.SettableSequenceId;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -81,7 +82,7 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
|
||||
@Override
|
||||
public ByteBuffer getValueShallowCopy() {
|
||||
return CellUtil.getValueBufferShallowCopy(ptSearcher.current());
|
||||
return PrivateCellUtil.getValueBufferShallowCopy(ptSearcher.current());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.column.ColumnReader;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.row.RowNodeReader;
|
||||
|
@ -420,7 +421,7 @@ public class PrefixTreeArrayScanner extends PrefixTreeCell implements CellScanne
|
|||
|
||||
protected int populateNonRowFieldsAndCompareTo(int cellNum, Cell key) {
|
||||
populateNonRowFields(cellNum);
|
||||
return CellUtil.compareKeyIgnoresMvcc(comparator, this, key);
|
||||
return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, this, key);
|
||||
}
|
||||
|
||||
protected void populateFirstNonRowFields() {
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.decode;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
|
||||
|
@ -91,7 +92,7 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
|||
}
|
||||
|
||||
//keep hunting for the rest of the row
|
||||
byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
|
||||
byte searchForByte = PrivateCellUtil.getRowByte(key, currentNodeDepth);
|
||||
fanIndex = currentRowNode.whichFanNode(searchForByte);
|
||||
if(fanIndex < 0){//no matching row. return early
|
||||
int insertionPoint = -fanIndex - 1;
|
||||
|
@ -140,7 +141,7 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
|||
}
|
||||
|
||||
//keep hunting for the rest of the row
|
||||
byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
|
||||
byte searchForByte = PrivateCellUtil.getRowByte(key, currentNodeDepth);
|
||||
fanIndex = currentRowNode.whichFanNode(searchForByte);
|
||||
if(fanIndex < 0){//no matching row. return early
|
||||
int insertionPoint = -fanIndex - 1;
|
||||
|
@ -287,7 +288,7 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
|||
if (i >= key.getRowLength()) {// key was shorter, so it's first
|
||||
return -1;
|
||||
}
|
||||
byte keyByte = CellUtil.getRowByte(key, i);
|
||||
byte keyByte = PrivateCellUtil.getRowByte(key, i);
|
||||
byte thisByte = rowBuffer[i];
|
||||
if (keyByte == thisByte) {
|
||||
continue;
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.ByteBufferCell;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.SettableSequenceId;
|
||||
|
@ -107,7 +107,7 @@ public class PrefixTreeCell extends ByteBufferCell implements SettableSequenceId
|
|||
return false;
|
||||
}
|
||||
// Temporary hack to maintain backwards compatibility with KeyValue.equals
|
||||
return CellUtil.equalsIgnoreMvccVersion(this, (Cell) obj);
|
||||
return PrivateCellUtil.equalsIgnoreMvccVersion(this, (Cell) obj);
|
||||
|
||||
// TODO return CellComparator.equals(this, (Cell)obj);//see HBASE-6907
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
|
||||
import org.apache.hadoop.hbase.codec.prefixtree.encode.column.ColumnSectionWriter;
|
||||
|
@ -274,7 +275,7 @@ public class PrefixTreeEncoder implements CellOutputStream {
|
|||
public void write(Cell cell) {
|
||||
ensurePerCellCapacities();
|
||||
|
||||
rowTokenizer.addSorted(CellUtil.fillRowRange(cell, rowRange));
|
||||
rowTokenizer.addSorted(PrivateCellUtil.fillRowRange(cell, rowRange));
|
||||
addFamilyPart(cell);
|
||||
addQualifierPart(cell);
|
||||
addTagPart(cell);
|
||||
|
@ -283,7 +284,7 @@ public class PrefixTreeEncoder implements CellOutputStream {
|
|||
|
||||
|
||||
private void addTagPart(Cell cell) {
|
||||
CellUtil.fillTagRange(cell, tagsRange);
|
||||
PrivateCellUtil.fillTagRange(cell, tagsRange);
|
||||
tagsDeduplicator.add(tagsRange);
|
||||
}
|
||||
|
||||
|
@ -329,13 +330,13 @@ public class PrefixTreeEncoder implements CellOutputStream {
|
|||
|
||||
private void addFamilyPart(Cell cell) {
|
||||
if (MULITPLE_FAMILIES_POSSIBLE || totalCells == 0) {
|
||||
CellUtil.fillFamilyRange(cell, familyRange);
|
||||
PrivateCellUtil.fillFamilyRange(cell, familyRange);
|
||||
familyDeduplicator.add(familyRange);
|
||||
}
|
||||
}
|
||||
|
||||
private void addQualifierPart(Cell cell) {
|
||||
CellUtil.fillQualifierRange(cell, qualifierRange);
|
||||
PrivateCellUtil.fillQualifierRange(cell, qualifierRange);
|
||||
qualifierDeduplicator.add(qualifierRange);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 (!CellUtil.matchingRow(lastKv, kv)) {
|
||||
if (!CellUtil.matchingRows(lastKv, kv)) {
|
||||
rowStartIndexes.add(i);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
|
@ -150,7 +151,7 @@ public class TestPrefixTreeSearcher {
|
|||
KeyValue kv = rows.getInputs().get(i);
|
||||
|
||||
//nextRow
|
||||
Cell inputNextRow = CellUtil.createFirstOnNextRow(kv);
|
||||
Cell inputNextRow = PrivateCellUtil.createFirstOnNextRow(kv);
|
||||
|
||||
CellScannerPosition position = beforeVsAfterOnMiss
|
||||
? searcher.positionAtOrBefore(inputNextRow)
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
|
@ -83,7 +84,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
|
|||
if (this.scanMetrics != null) {
|
||||
long resultSize = 0;
|
||||
for (Cell cell : values) {
|
||||
resultSize += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
resultSize += PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
}
|
||||
this.scanMetrics.countOfBytesInResults.addAndGet(resultSize);
|
||||
this.scanMetrics.countOfRowsScanned.incrementAndGet();
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -128,7 +129,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
|
|||
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) {
|
||||
continue;
|
||||
}
|
||||
if (CellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(), hri.getStartKey(),
|
||||
if (PrivateCellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(), hri.getStartKey(),
|
||||
hri.getEndKey())) {
|
||||
regions.add(hri);
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -210,11 +211,11 @@ public class HalfStoreFileReader extends StoreFileReader {
|
|||
@Override
|
||||
public int seekTo(Cell key) throws IOException {
|
||||
if (top) {
|
||||
if (CellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) {
|
||||
return -1;
|
||||
}
|
||||
} else {
|
||||
if (CellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), 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);
|
||||
|
@ -235,11 +236,11 @@ public class HalfStoreFileReader extends StoreFileReader {
|
|||
// except
|
||||
// that we call reseekTo (and not seekTo) on the delegate.
|
||||
if (top) {
|
||||
if (CellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) {
|
||||
return -1;
|
||||
}
|
||||
} else {
|
||||
if (CellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), 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);
|
||||
|
@ -261,13 +262,13 @@ public class HalfStoreFileReader extends StoreFileReader {
|
|||
public boolean seekBefore(Cell key) throws IOException {
|
||||
if (top) {
|
||||
Optional<Cell> fk = getFirstKey();
|
||||
if (CellUtil.compareKeyIgnoresMvcc(getComparator(), key, fk.get()) <= 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, fk.get()) <= 0) {
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
// The equals sign isn't strictly necessary just here to be consistent
|
||||
// with seekTo
|
||||
if (CellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
|
||||
boolean ret = this.delegate.seekBefore(splitCell);
|
||||
if (ret) {
|
||||
atEnd = false;
|
||||
|
|
|
@ -28,6 +28,7 @@ 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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
|
@ -177,7 +178,8 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
|
|||
firstKeyInChunk = CellUtil.copyRow(cell);
|
||||
} else {
|
||||
firstKeyInChunk =
|
||||
CellUtil.getCellKeySerializedAsKeyValueKey(CellUtil.createFirstOnRowCol(cell));
|
||||
PrivateCellUtil
|
||||
.getCellKeySerializedAsKeyValueKey(PrivateCellUtil.createFirstOnRowCol(cell));
|
||||
}
|
||||
allocateNewChunk();
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparator;
|
||||
//import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KeyOnlyKeyValue;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -265,7 +266,7 @@ public class HFileBlockIndex {
|
|||
|
||||
// Adding blockKeys
|
||||
for (Cell key : blockKeys) {
|
||||
heapSize += ClassSize.align(CellUtil.estimatedHeapSizeOf(key));
|
||||
heapSize += ClassSize.align(PrivateCellUtil.estimatedHeapSizeOf(key));
|
||||
}
|
||||
}
|
||||
// Add comparator and the midkey atomicreference
|
||||
|
@ -767,7 +768,7 @@ public class HFileBlockIndex {
|
|||
// TODO avoid array call.
|
||||
nonRootIndex.asSubByteBuffer(midKeyOffset, midLength, pair);
|
||||
nonRootIndexkeyOnlyKV.setKey(pair.getFirst(), pair.getSecond(), midLength);
|
||||
int cmp = CellUtil.compareKeyIgnoresMvcc(comparator, key, nonRootIndexkeyOnlyKV);
|
||||
int cmp = PrivateCellUtil.compareKeyIgnoresMvcc(comparator, key, nonRootIndexkeyOnlyKV);
|
||||
|
||||
// key lives above the midpoint
|
||||
if (cmp > 0)
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -322,7 +323,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
if (this.isSeekToRow) {
|
||||
// seek to the first kv on this row
|
||||
shouldScanKeysValues =
|
||||
(scanner.seekTo(CellUtil.createFirstOnRow(this.row)) != -1);
|
||||
(scanner.seekTo(PrivateCellUtil.createFirstOnRow(this.row)) != -1);
|
||||
} else {
|
||||
shouldScanKeysValues = scanner.seekTo();
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedKeyValue;
|
||||
|
@ -728,7 +729,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
offsetFromPos += Bytes.SIZEOF_LONG;
|
||||
blockBuffer.asSubByteBuffer(blockBuffer.position() + offsetFromPos, klen, pair);
|
||||
bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), klen);
|
||||
int comp = CellUtil.compareKeyIgnoresMvcc(reader.getComparator(), key, bufBackedKeyOnlyKv);
|
||||
int comp =
|
||||
PrivateCellUtil.compareKeyIgnoresMvcc(reader.getComparator(), key, bufBackedKeyOnlyKv);
|
||||
offsetFromPos += klen + vlen;
|
||||
if (this.reader.getFileContext().isIncludesTags()) {
|
||||
// Read short as unsigned, high byte first
|
||||
|
@ -811,7 +813,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
} else {
|
||||
// The comparison with no_next_index_key has to be checked
|
||||
if (this.nextIndexedKey != null &&
|
||||
(this.nextIndexedKey == KeyValueScanner.NO_NEXT_INDEXED_KEY || CellUtil
|
||||
(this.nextIndexedKey == KeyValueScanner.NO_NEXT_INDEXED_KEY || PrivateCellUtil
|
||||
.compareKeyIgnoresMvcc(reader.getComparator(), key, nextIndexedKey) < 0)) {
|
||||
// The reader shall continue to scan the current data block instead
|
||||
// of querying the
|
||||
|
@ -865,7 +867,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
return false;
|
||||
}
|
||||
Cell firstKey = getFirstKeyCellInBlock(seekToBlock);
|
||||
if (CellUtil.compareKeyIgnoresMvcc(reader.getComparator(), firstKey, key) >= 0) {
|
||||
if (PrivateCellUtil.compareKeyIgnoresMvcc(reader.getComparator(), firstKey, key) >= 0) {
|
||||
long previousBlockOffset = seekToBlock.getPrevBlockOffset();
|
||||
// The key we are interested in
|
||||
if (previousBlockOffset == -1) {
|
||||
|
@ -1229,7 +1231,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
public int compareKey(CellComparator comparator, Cell key) {
|
||||
blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen, pair);
|
||||
this.bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), currKeyLen);
|
||||
return CellUtil.compareKeyIgnoresMvcc(comparator, key, this.bufBackedKeyOnlyKv);
|
||||
return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, key, this.bufBackedKeyOnlyKv);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl.MetaCellComparator;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -239,7 +240,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
throw new IOException("Key cannot be null or empty");
|
||||
}
|
||||
if (lastCell != null) {
|
||||
int keyComp = CellUtil.compareKeyIgnoresMvcc(comparator, lastCell, cell);
|
||||
int keyComp = PrivateCellUtil.compareKeyIgnoresMvcc(comparator, lastCell, cell);
|
||||
|
||||
if (keyComp > 0) {
|
||||
throw new IOException("Added a key not lexically larger than"
|
||||
|
@ -341,7 +342,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
int onDiskSize = blockWriter.getOnDiskSizeWithHeader();
|
||||
Cell indexEntry =
|
||||
getMidpoint(this.comparator, lastCellOfPreviousBlock, firstCellInBlock);
|
||||
dataBlockIndexWriter.addEntry(CellUtil.getCellKeySerializedAsKeyValueKey(indexEntry),
|
||||
dataBlockIndexWriter.addEntry(PrivateCellUtil.getCellKeySerializedAsKeyValueKey(indexEntry),
|
||||
lastDataBlockOffset, onDiskSize);
|
||||
totalUncompressedBytes += blockWriter.getUncompressedSizeWithHeader();
|
||||
if (cacheConf.shouldCacheDataOnWrite()) {
|
||||
|
@ -397,7 +398,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
}
|
||||
// If midRow is null, just return 'right'. Can't do optimization.
|
||||
if (midRow == null) return right;
|
||||
return CellUtil.createFirstOnRow(midRow);
|
||||
return PrivateCellUtil.createFirstOnRow(midRow);
|
||||
}
|
||||
// Rows are same. Compare on families.
|
||||
diff = comparator.compareFamilies(left, right);
|
||||
|
@ -419,7 +420,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
// 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.createFirstOnRowFamily(right, midRow, 0, midRow.length);
|
||||
return PrivateCellUtil.createFirstOnRowFamily(right, midRow, 0, midRow.length);
|
||||
}
|
||||
// Families are same. Compare on qualifiers.
|
||||
diff = comparator.compareQualifiers(left, right);
|
||||
|
@ -441,7 +442,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
// 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.createFirstOnRowCol(right, midRow, 0, midRow.length);
|
||||
return PrivateCellUtil.createFirstOnRowCol(right, midRow, 0, midRow.length);
|
||||
}
|
||||
// No opportunity for optimization. Just return right key.
|
||||
return right;
|
||||
|
@ -738,7 +739,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
|
||||
blockWriter.write(cell);
|
||||
|
||||
totalKeyLength += CellUtil.estimatedSerializedSizeOfKey(cell);
|
||||
totalKeyLength += PrivateCellUtil.estimatedSerializedSizeOfKey(cell);
|
||||
totalValueLength += cell.getValueLength();
|
||||
|
||||
// Are we the first key in this block?
|
||||
|
@ -776,7 +777,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
if (lastCell != null) {
|
||||
// Make a copy. The copy is stuffed into our fileinfo map. Needs a clean
|
||||
// byte buffer. Won't take a tuple.
|
||||
byte [] lastKey = CellUtil.getCellKeySerializedAsKeyValueKey(this.lastCell);
|
||||
byte [] lastKey = PrivateCellUtil.getCellKeySerializedAsKeyValueKey(this.lastCell);
|
||||
fileInfo.append(FileInfo.LASTKEY, lastKey, false);
|
||||
}
|
||||
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.CellSink;
|
||||
|
@ -248,7 +249,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
|||
Cell mobCell = mobStore.resolve(c, false);
|
||||
if (mobCell.getValueLength() != 0) {
|
||||
// put the mob data back to the store file
|
||||
CellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||
PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||
writer.append(mobCell);
|
||||
cellsCountCompactedFromMob++;
|
||||
cellsSizeCompactedFromMob += mobCell.getValueLength();
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
|
@ -175,7 +176,7 @@ public final class MobUtils {
|
|||
*/
|
||||
public static boolean isMobReferenceCell(Cell cell) {
|
||||
if (cell.getTagsLength() > 0) {
|
||||
Tag tag = CellUtil.getTag(cell, TagType.MOB_REFERENCE_TAG_TYPE);
|
||||
Tag tag = PrivateCellUtil.getTag(cell, TagType.MOB_REFERENCE_TAG_TYPE);
|
||||
return tag != null;
|
||||
}
|
||||
return false;
|
||||
|
@ -188,7 +189,7 @@ public final class MobUtils {
|
|||
*/
|
||||
public static Tag getTableNameTag(Cell cell) {
|
||||
if (cell.getTagsLength() > 0) {
|
||||
return CellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE);
|
||||
return PrivateCellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -501,7 +502,7 @@ public final class MobUtils {
|
|||
|
||||
public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTags) {
|
||||
byte[] refValue = Bytes.add(Bytes.toBytes(cell.getValueLength()), fileName);
|
||||
return CellUtil.createCell(cell, refValue, TagUtil.concatTags(refCellTags, cell));
|
||||
return PrivateCellUtil.createCell(cell, refValue, TagUtil.concatTags(refCellTags, cell));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -761,7 +762,7 @@ public final class MobUtils {
|
|||
* @return The real mob value length.
|
||||
*/
|
||||
public static int getMobValueLength(Cell cell) {
|
||||
return CellUtil.getValueAsInt(cell);
|
||||
return PrivateCellUtil.getValueAsInt(cell);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -891,7 +892,7 @@ public final class MobUtils {
|
|||
* @return A delete marker with the ref tag.
|
||||
*/
|
||||
public static Cell createMobRefDeleteMarker(Cell cell) {
|
||||
return CellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell));
|
||||
return PrivateCellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,8 +31,8 @@ import java.util.UUID;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.SizedCellScanner;
|
||||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||
|
@ -146,7 +146,7 @@ public class ReplicationProtbufUtil {
|
|||
List<Cell> cells = edit.getCells();
|
||||
// Add up the size. It is used later serializing out the kvs.
|
||||
for (Cell cell: cells) {
|
||||
size += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
size += PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
}
|
||||
// Collect up the cells
|
||||
allCells.add(cells);
|
||||
|
|
|
@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
|
@ -2958,7 +2959,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
Cell cell = cells.get(i);
|
||||
// Check if time is LATEST, change to time of most recent addition if so
|
||||
// This is expensive.
|
||||
if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP && CellUtil.isDeleteType(cell)) {
|
||||
if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP
|
||||
&& PrivateCellUtil.isDeleteType(cell)) {
|
||||
byte[] qual = CellUtil.cloneQualifier(cell);
|
||||
|
||||
Integer count = kvCount.get(qual);
|
||||
|
@ -2981,7 +2983,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
|
||||
}
|
||||
} else {
|
||||
CellUtil.updateLatestStamp(cell, byteNow, 0);
|
||||
PrivateCellUtil.updateLatestStamp(cell, byteNow, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2993,14 +2995,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
if (result.size() < count) {
|
||||
// Nothing to delete
|
||||
CellUtil.updateLatestStamp(cell, byteNow, 0);
|
||||
PrivateCellUtil.updateLatestStamp(cell, byteNow, 0);
|
||||
return;
|
||||
}
|
||||
if (result.size() > count) {
|
||||
throw new RuntimeException("Unexpected size: " + result.size());
|
||||
}
|
||||
Cell getCell = result.get(count - 1);
|
||||
CellUtil.setTimestamp(cell, getCell.getTimestamp());
|
||||
PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -3753,7 +3755,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
} else if (result.size() == 1 && !valueIsNull) {
|
||||
Cell kv = result.get(0);
|
||||
cellTs = kv.getTimestamp();
|
||||
int compareResult = CellUtil.compareValue(kv, comparator);
|
||||
int compareResult = PrivateCellUtil.compareValue(kv, comparator);
|
||||
matches = matches(op, compareResult);
|
||||
}
|
||||
// If matches put the new put or delete the new delete
|
||||
|
@ -3876,7 +3878,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
for (List<Cell> cells: cellItr) {
|
||||
if (cells == null) return;
|
||||
for (Cell cell : cells) {
|
||||
CellUtil.setSequenceId(cell, sequenceId);
|
||||
PrivateCellUtil.setSequenceId(cell, sequenceId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3894,7 +3896,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
assert cells instanceof RandomAccess;
|
||||
int listSize = cells.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
CellUtil.updateLatestStamp(cells.get(i), now, 0);
|
||||
PrivateCellUtil.updateLatestStamp(cells.get(i), now, 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3919,7 +3921,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
List<Tag> newTags = TagUtil.carryForwardTags(null, cell);
|
||||
newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL());
|
||||
// Rewrite the cell with the updated set of tags
|
||||
cells.set(i, CellUtil.createCell(cell, newTags));
|
||||
cells.set(i, PrivateCellUtil.createCell(cell, newTags));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -4379,7 +4381,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
skippedEdits++;
|
||||
continue;
|
||||
}
|
||||
CellUtil.setSequenceId(cell, currentReplaySeqId);
|
||||
PrivateCellUtil.setSequenceId(cell, currentReplaySeqId);
|
||||
|
||||
restoreEdit(store, cell, memstoreSize);
|
||||
editsCount++;
|
||||
|
@ -6147,7 +6149,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @return true When there are more cells in the row to be read
|
||||
*/
|
||||
private boolean moreCellsInRow(final Cell nextKv, Cell currentRowCell) {
|
||||
return nextKv != null && CellUtil.matchingRow(nextKv, currentRowCell);
|
||||
return nextKv != null && CellUtil.matchingRows(nextKv, currentRowCell);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -6298,8 +6300,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
scannerContext.setTimeProgress(timeProgress);
|
||||
scannerContext.incrementBatchProgress(results.size());
|
||||
for (Cell cell : results) {
|
||||
scannerContext.incrementSizeProgress(CellUtil.estimatedSerializedSizeOf(cell),
|
||||
CellUtil.estimatedHeapSizeOf(cell));
|
||||
scannerContext.incrementSizeProgress(PrivateCellUtil.estimatedSerializedSizeOf(cell),
|
||||
PrivateCellUtil.estimatedHeapSizeOf(cell));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6388,17 +6390,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
throws IOException {
|
||||
Cell nextJoinedKv = joinedHeap.peek();
|
||||
boolean matchCurrentRow =
|
||||
nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv, currentRowCell);
|
||||
nextJoinedKv != null && CellUtil.matchingRows(nextJoinedKv, currentRowCell);
|
||||
boolean matchAfterSeek = false;
|
||||
|
||||
// If the next value in the joined heap does not match the current row, try to seek to the
|
||||
// correct row
|
||||
if (!matchCurrentRow) {
|
||||
Cell firstOnCurrentRow = CellUtil.createFirstOnRow(currentRowCell);
|
||||
Cell firstOnCurrentRow = PrivateCellUtil.createFirstOnRow(currentRowCell);
|
||||
boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true);
|
||||
matchAfterSeek =
|
||||
seekSuccessful && joinedHeap.peek() != null
|
||||
&& CellUtil.matchingRow(joinedHeap.peek(), currentRowCell);
|
||||
&& CellUtil.matchingRows(joinedHeap.peek(), currentRowCell);
|
||||
}
|
||||
|
||||
return matchCurrentRow || matchAfterSeek;
|
||||
|
@ -6426,7 +6428,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
assert this.joinedContinuationRow == null: "Trying to go to next row during joinedHeap read.";
|
||||
Cell next;
|
||||
while ((next = this.storeHeap.peek()) != null &&
|
||||
CellUtil.matchingRow(next, curRowCell)) {
|
||||
CellUtil.matchingRows(next, curRowCell)) {
|
||||
this.storeHeap.next(MOCKED_LIST);
|
||||
}
|
||||
resetFilters();
|
||||
|
@ -6474,7 +6476,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
boolean result = false;
|
||||
startRegionOperation();
|
||||
Cell kv = CellUtil.createFirstOnRow(row, 0, (short) row.length);
|
||||
Cell kv = PrivateCellUtil.createFirstOnRow(row, 0, (short) row.length);
|
||||
try {
|
||||
// use request seek to make use of the lazy seek option. See HBASE-5520
|
||||
result = this.storeHeap.requestSeek(kv, true, true);
|
||||
|
@ -7146,7 +7148,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
if (walEdit.isEmpty()) {
|
||||
// If walEdit is empty, we put nothing in WAL. WAL stamps Cells with sequence id.
|
||||
// If no WAL, need to stamp it here.
|
||||
CellUtil.setSequenceId(cell, sequenceId);
|
||||
PrivateCellUtil.setSequenceId(cell, sequenceId);
|
||||
}
|
||||
applyToMemStore(getStore(cell), cell, memstoreAccounting);
|
||||
}
|
||||
|
@ -7565,8 +7567,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
.setTags(TagUtil.fromList(tags))
|
||||
.build();
|
||||
} else {
|
||||
CellUtil.updateLatestStamp(delta, now);
|
||||
return CollectionUtils.isEmpty(tags) ? delta : CellUtil.createCell(delta, tags);
|
||||
PrivateCellUtil.updateLatestStamp(delta, now);
|
||||
return CollectionUtils.isEmpty(tags) ? delta : PrivateCellUtil.createCell(delta, tags);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -7579,7 +7581,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// throw DoNotRetryIOException instead of IllegalArgumentException
|
||||
throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
|
||||
}
|
||||
return CellUtil.getValueAsLong(cell);
|
||||
return PrivateCellUtil.getValueAsLong(cell);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -40,8 +40,8 @@ import org.apache.hadoop.fs.LocatedFileStatus;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
|
@ -666,7 +666,7 @@ public class HRegionFileSystem {
|
|||
try {
|
||||
if (top) {
|
||||
//check if larger than last key.
|
||||
Cell splitKey = CellUtil.createFirstOnRow(splitRow);
|
||||
Cell splitKey = PrivateCellUtil.createFirstOnRow(splitRow);
|
||||
Optional<Cell> lastKey = f.getLastKey();
|
||||
// If lastKey is null means storefile is empty.
|
||||
if (!lastKey.isPresent()) {
|
||||
|
@ -677,7 +677,7 @@ public class HRegionFileSystem {
|
|||
}
|
||||
} else {
|
||||
//check if smaller than first key
|
||||
Cell splitKey = CellUtil.createLastOnRow(splitRow);
|
||||
Cell splitKey = PrivateCellUtil.createLastOnRow(splitRow);
|
||||
Optional<Cell> firstKey = f.getFirstKey();
|
||||
// If firstKey is null means storefile is empty.
|
||||
if (!firstKey.isPresent()) {
|
||||
|
|
|
@ -25,6 +25,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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
|
@ -60,7 +61,7 @@ public class MutableSegment extends Segment {
|
|||
|
||||
// Get the Cells for the row/family/qualifier regardless of timestamp.
|
||||
// For this case we want to clean up any other puts
|
||||
Cell firstCell = CellUtil.createFirstOnRowColTS(cell, HConstants.LATEST_TIMESTAMP);
|
||||
Cell firstCell = PrivateCellUtil.createFirstOnRowColTS(cell, HConstants.LATEST_TIMESTAMP);
|
||||
SortedSet<Cell> ss = this.tailSet(firstCell);
|
||||
Iterator<Cell> it = ss.iterator();
|
||||
// versions visible to oldest scanner
|
||||
|
|
|
@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
|
|||
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.MultiActionResultTooLarge;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
@ -930,7 +931,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
if (r.maxCellSize > 0) {
|
||||
CellScanner cells = m.cellScanner();
|
||||
while (cells.advance()) {
|
||||
int size = CellUtil.estimatedSerializedSizeOf(cells.current());
|
||||
int size = PrivateCellUtil.estimatedSerializedSizeOf(cells.current());
|
||||
if (size > r.maxCellSize) {
|
||||
String msg = "Cell with size " + size + " exceeds limit of " + r.maxCellSize + " bytes";
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -1300,7 +1301,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
Object addSize(RpcCallContext context, Result r, Object lastBlock) {
|
||||
if (context != null && r != null && !r.isEmpty()) {
|
||||
for (Cell c : r.rawCells()) {
|
||||
context.incrementResponseCellSize(CellUtil.estimatedSerializedSizeOf(c));
|
||||
context.incrementResponseCellSize(PrivateCellUtil.estimatedSerializedSizeOf(c));
|
||||
|
||||
// Since byte buffers can point all kinds of crazy places it's harder to keep track
|
||||
// of which blocks are kept alive by what byte buffer.
|
||||
|
@ -3061,7 +3062,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
// so then we need to increase the numOfCompleteRows.
|
||||
if (results.isEmpty()) {
|
||||
if (rsh.rowOfLastPartialResult != null &&
|
||||
!CellUtil.matchingRow(values.get(0), rsh.rowOfLastPartialResult)) {
|
||||
!CellUtil.matchingRows(values.get(0), rsh.rowOfLastPartialResult)) {
|
||||
numOfCompleteRows++;
|
||||
checkLimitOfRows(numOfCompleteRows, limitOfRows, moreRows, scannerContext,
|
||||
builder);
|
||||
|
@ -3069,7 +3070,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
} else {
|
||||
Result lastResult = results.get(results.size() - 1);
|
||||
if (lastResult.mayHaveMoreCellsInRow() &&
|
||||
!CellUtil.matchingRow(values.get(0), lastResult.getRow())) {
|
||||
!CellUtil.matchingRows(values.get(0), lastResult.getRow())) {
|
||||
numOfCompleteRows++;
|
||||
checkLimitOfRows(numOfCompleteRows, limitOfRows, moreRows, scannerContext,
|
||||
builder);
|
||||
|
|
|
@ -107,7 +107,7 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
|
|||
KeyValueScanner scanner;
|
||||
while ((scanner = heap.poll()) != null) {
|
||||
Cell topKey = scanner.peek();
|
||||
if ((CellUtil.matchingRow(seekKey, topKey) && comparator
|
||||
if ((CellUtil.matchingRows(seekKey, topKey) && comparator
|
||||
.getComparator().compare(seekKey, topKey) <= 0)
|
||||
|| comparator.getComparator().compareRows(seekKey, topKey) > 0) {
|
||||
heap.add(scanner);
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.util.List;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
|
@ -73,7 +74,7 @@ class ReversedRegionScannerImpl extends RegionScannerImpl {
|
|||
protected boolean nextRow(ScannerContext scannerContext, Cell curRowCell)
|
||||
throws IOException {
|
||||
assert super.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read.";
|
||||
this.storeHeap.seekToPreviousRow(CellUtil.createFirstOnRow(curRowCell));
|
||||
this.storeHeap.seekToPreviousRow(PrivateCellUtil.createFirstOnRow(curRowCell));
|
||||
resetFilters();
|
||||
// Calling the hook in CP which allows it to do a fast forward
|
||||
if (this.region.getCoprocessorHost() != null) {
|
||||
|
|
|
@ -71,7 +71,7 @@ public class ReversedStoreScanner extends StoreScanner implements KeyValueScanne
|
|||
throws IOException {
|
||||
// Seek all scanners to the start of the Row (or if the exact matching row
|
||||
// key does not exist, then to the start of the previous matching Row).
|
||||
if (CellUtil.matchingRow(seekKey, HConstants.EMPTY_START_ROW)) {
|
||||
if (CellUtil.matchingRows(seekKey, HConstants.EMPTY_START_ROW)) {
|
||||
for (KeyValueScanner scanner : scanners) {
|
||||
scanner.seekToLastRow();
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
|
@ -318,7 +318,7 @@ public abstract class Segment {
|
|||
protected long heapSizeChange(Cell cell, boolean succ) {
|
||||
if (succ) {
|
||||
return ClassSize
|
||||
.align(indexEntrySize() + CellUtil.estimatedHeapSizeOf(cell));
|
||||
.align(indexEntrySize() + PrivateCellUtil.estimatedHeapSizeOf(cell));
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.SortedSet;
|
|||
import org.apache.commons.lang3.NotImplementedException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
||||
|
@ -205,14 +205,14 @@ public class SegmentScanner implements KeyValueScanner {
|
|||
boolean keepSeeking;
|
||||
Cell key = cell;
|
||||
do {
|
||||
Cell firstKeyOnRow = CellUtil.createFirstOnRow(key);
|
||||
Cell firstKeyOnRow = PrivateCellUtil.createFirstOnRow(key);
|
||||
SortedSet<Cell> cellHead = segment.headSet(firstKeyOnRow);
|
||||
Cell lastCellBeforeRow = cellHead.isEmpty() ? null : cellHead.last();
|
||||
if (lastCellBeforeRow == null) {
|
||||
current = null;
|
||||
return false;
|
||||
}
|
||||
Cell firstKeyOnPreviousRow = CellUtil.createFirstOnRow(lastCellBeforeRow);
|
||||
Cell firstKeyOnPreviousRow = PrivateCellUtil.createFirstOnRow(lastCellBeforeRow);
|
||||
this.stopSkippingKVsIfNextRow = true;
|
||||
seek(firstKeyOnPreviousRow);
|
||||
this.stopSkippingKVsIfNextRow = false;
|
||||
|
@ -243,7 +243,7 @@ public class SegmentScanner implements KeyValueScanner {
|
|||
return false;
|
||||
}
|
||||
|
||||
Cell firstCellOnLastRow = CellUtil.createFirstOnRow(higherCell);
|
||||
Cell firstCellOnLastRow = PrivateCellUtil.createFirstOnRow(higherCell);
|
||||
|
||||
if (seek(firstCellOnLastRow)) {
|
||||
return true;
|
||||
|
|
|
@ -36,9 +36,9 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
|
@ -253,7 +253,7 @@ public class StoreFileReader {
|
|||
if (columns != null && columns.size() == 1) {
|
||||
byte[] column = columns.first();
|
||||
// create the required fake key
|
||||
Cell kvKey = CellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);
|
||||
Cell kvKey = PrivateCellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);
|
||||
return passesGeneralRowColBloomFilter(kvKey);
|
||||
}
|
||||
|
||||
|
@ -336,7 +336,7 @@ public class StoreFileReader {
|
|||
if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
|
||||
kvKey = cell;
|
||||
} else {
|
||||
kvKey = CellUtil.createFirstOnRowCol(cell);
|
||||
kvKey = PrivateCellUtil.createFirstOnRowCol(cell);
|
||||
}
|
||||
return checkGeneralBloomFilter(null, kvKey, bloomFilter);
|
||||
}
|
||||
|
@ -381,7 +381,7 @@ public class StoreFileReader {
|
|||
// columns, a file might be skipped if using row+col Bloom filter.
|
||||
// In order to ensure this file is included an additional check is
|
||||
// required looking only for a row bloom.
|
||||
Cell rowBloomKey = CellUtil.createFirstOnRow(kvKey);
|
||||
Cell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey);
|
||||
// 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
|
||||
|
|
|
@ -32,9 +32,9 @@ import java.util.concurrent.atomic.LongAdder;
|
|||
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.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -262,7 +262,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
protected void setCurrentCell(Cell newVal) throws IOException {
|
||||
this.cur = newVal;
|
||||
if (this.cur != null && this.reader.isBulkLoaded() && !this.reader.isSkipResetSeqId()) {
|
||||
CellUtil.setSequenceId(cur, this.reader.getSequenceID());
|
||||
PrivateCellUtil.setSequenceId(cur, this.reader.getSequenceID());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -381,7 +381,8 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
if (reader.getBloomFilterType() == BloomType.ROWCOL) {
|
||||
haveToSeek = reader.passesGeneralRowColBloomFilter(kv);
|
||||
} else if (canOptimizeForNonNullColumn
|
||||
&& ((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) {
|
||||
&& ((PrivateCellUtil.isDeleteFamily(kv)
|
||||
|| PrivateCellUtil.isDeleteFamilyVersion(kv)))) {
|
||||
// if there is no such delete family kv in the store file,
|
||||
// then no need to seek.
|
||||
haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(), kv.getRowOffset(),
|
||||
|
@ -405,7 +406,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
// a higher timestamp than the max timestamp in this file. We know that
|
||||
// the next point when we have to consider this file again is when we
|
||||
// pass the max timestamp of this file (with the same row/column).
|
||||
setCurrentCell(CellUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
|
||||
setCurrentCell(PrivateCellUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
|
||||
} else {
|
||||
// This will be the case e.g. when we need to seek to the next
|
||||
// row/column, and we don't know exactly what they are, so we set the
|
||||
|
@ -423,7 +424,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
// key/value and the store scanner will progress to the next column. This
|
||||
// is obviously not a "real real" seek, but unlike the fake KV earlier in
|
||||
// this method, we want this to be propagated to ScanQueryMatcher.
|
||||
setCurrentCell(CellUtil.createLastOnRowCol(kv));
|
||||
setCurrentCell(PrivateCellUtil.createLastOnRowCol(kv));
|
||||
|
||||
realSeekDone = true;
|
||||
return true;
|
||||
|
@ -492,14 +493,14 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
boolean keepSeeking = false;
|
||||
Cell key = originalKey;
|
||||
do {
|
||||
Cell seekKey = CellUtil.createFirstOnRow(key);
|
||||
Cell seekKey = PrivateCellUtil.createFirstOnRow(key);
|
||||
if (seekCount != null) seekCount.increment();
|
||||
if (!hfs.seekBefore(seekKey)) {
|
||||
this.cur = null;
|
||||
return false;
|
||||
}
|
||||
Cell curCell = hfs.getCell();
|
||||
Cell firstKeyOfPreviousRow = CellUtil.createFirstOnRow(curCell);
|
||||
Cell firstKeyOfPreviousRow = PrivateCellUtil.createFirstOnRow(curCell);
|
||||
|
||||
if (seekCount != null) seekCount.increment();
|
||||
if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
|
||||
|
@ -542,7 +543,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
if (!lastRow.isPresent()) {
|
||||
return false;
|
||||
}
|
||||
Cell seekKey = CellUtil.createFirstOnRow(lastRow.get());
|
||||
Cell seekKey = PrivateCellUtil.createFirstOnRow(lastRow.get());
|
||||
if (seek(seekKey)) {
|
||||
return true;
|
||||
} else {
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -218,7 +219,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
|||
|
||||
private void appendDeleteFamilyBloomFilter(final Cell cell)
|
||||
throws IOException {
|
||||
if (!CellUtil.isDeleteFamily(cell) && !CellUtil.isDeleteFamilyVersion(cell)) {
|
||||
if (!PrivateCellUtil.isDeleteFamily(cell) && !PrivateCellUtil.isDeleteFamilyVersion(cell)) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
|
|
@ -35,6 +35,7 @@ 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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.client.IsolationLevel;
|
||||
|
@ -397,7 +398,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
scanner.seek(seekKey);
|
||||
Cell c = scanner.peek();
|
||||
if (c != null) {
|
||||
totalScannersSoughtBytes += CellUtil.estimatedSerializedSizeOf(c);
|
||||
totalScannersSoughtBytes += PrivateCellUtil.estimatedSerializedSizeOf(c);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
|
@ -568,7 +569,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
++kvsScanned;
|
||||
}
|
||||
checkScanOrder(prevCell, cell, comparator);
|
||||
int cellSize = CellUtil.estimatedSerializedSizeOf(cell);
|
||||
int cellSize = PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
bytesRead += cellSize;
|
||||
prevCell = cell;
|
||||
scannerContext.setLastPeekedCell(cell);
|
||||
|
@ -606,7 +607,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
totalBytesRead += cellSize;
|
||||
|
||||
// Update the progress of the scanner context
|
||||
scannerContext.incrementSizeProgress(cellSize, CellUtil.estimatedHeapSizeOf(cell));
|
||||
scannerContext.incrementSizeProgress(cellSize,
|
||||
PrivateCellUtil.estimatedHeapSizeOf(cell));
|
||||
scannerContext.incrementBatchProgress(1);
|
||||
|
||||
if (matcher.isUserScan() && totalBytesRead > maxRowSize) {
|
||||
|
@ -940,7 +942,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
}
|
||||
|
||||
protected boolean seekToNextRow(Cell c) throws IOException {
|
||||
return reseek(CellUtil.createLastOnRow(c));
|
||||
return reseek(PrivateCellUtil.createLastOnRow(c));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -29,6 +29,7 @@ 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.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -297,7 +298,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter {
|
|||
sanityCheckLeft(left, cell);
|
||||
doCreateWriter = true;
|
||||
} else if (lastRowInCurrentWriter != null
|
||||
&& !CellUtil.matchingRow(cell, lastRowInCurrentWriter, 0,
|
||||
&& !PrivateCellUtil.matchingRows(cell, lastRowInCurrentWriter, 0,
|
||||
lastRowInCurrentWriter.length)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Stopping to use a writer after [" + Bytes.toString(lastRowInCurrentWriter)
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -102,7 +103,7 @@ public abstract class TimeRangeTracker {
|
|||
*/
|
||||
public void includeTimestamp(final Cell cell) {
|
||||
includeTimestamp(cell.getTimestamp());
|
||||
if (CellUtil.isDeleteColumnOrFamily(cell)) {
|
||||
if (PrivateCellUtil.isDeleteColumnOrFamily(cell)) {
|
||||
includeTimestamp(0);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,8 +33,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
|
@ -390,7 +390,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||
if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) {
|
||||
lastCleanCell = c;
|
||||
lastCleanCellSeqId = c.getSequenceId();
|
||||
CellUtil.setSequenceId(c, 0);
|
||||
PrivateCellUtil.setSequenceId(c, 0);
|
||||
} else {
|
||||
lastCleanCell = null;
|
||||
lastCleanCellSeqId = 0;
|
||||
|
@ -420,7 +420,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||
// HBASE-16931, set back sequence id to avoid affecting scan order unexpectedly.
|
||||
// ShipperListener will do a clone of the last cells it refer, so need to set back
|
||||
// sequence id before ShipperListener.beforeShipped
|
||||
CellUtil.setSequenceId(lastCleanCell, lastCleanCellSeqId);
|
||||
PrivateCellUtil.setSequenceId(lastCleanCell, lastCleanCellSeqId);
|
||||
}
|
||||
// Clone the cells that are in the writer so that they are freed of references,
|
||||
// if they are holding any.
|
||||
|
@ -437,7 +437,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||
}
|
||||
if (lastCleanCell != null) {
|
||||
// HBASE-16931, set back sequence id to avoid affecting scan order unexpectedly
|
||||
CellUtil.setSequenceId(lastCleanCell, lastCleanCellSeqId);
|
||||
PrivateCellUtil.setSequenceId(lastCleanCell, lastCleanCellSeqId);
|
||||
}
|
||||
// Log the progress of long running compactions every minute if
|
||||
// logging at DEBUG level
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.NavigableSet;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode;
|
||||
|
||||
|
@ -103,7 +104,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
|
|||
public ScanQueryMatcher.MatchCode checkColumn(Cell cell, byte type) {
|
||||
// delete markers should never be passed to an
|
||||
// *Explicit*ColumnTracker
|
||||
assert !CellUtil.isDelete(type);
|
||||
assert !PrivateCellUtil.isDelete(type);
|
||||
do {
|
||||
// No more columns left, we are done with this query
|
||||
if (done()) {
|
||||
|
@ -152,7 +153,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
|
|||
@Override
|
||||
public ScanQueryMatcher.MatchCode checkVersions(Cell cell, long timestamp, byte type,
|
||||
boolean ignoreCount) throws IOException {
|
||||
assert !CellUtil.isDelete(type);
|
||||
assert !PrivateCellUtil.isDelete(type);
|
||||
if (ignoreCount) {
|
||||
return ScanQueryMatcher.MatchCode.INCLUDE;
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
||||
|
||||
|
@ -58,7 +59,7 @@ public class MajorCompactionScanQueryMatcher extends DropDeletesCompactionScanQu
|
|||
// 7. Delete marker need to be version counted together with puts
|
||||
// they affect
|
||||
//
|
||||
if (CellUtil.isDelete(typeByte)) {
|
||||
if (PrivateCellUtil.isDelete(typeByte)) {
|
||||
if (mvccVersion > maxReadPointToTrackVersions) {
|
||||
// We can not drop this delete marker yet, and also we should not use this delete marker to
|
||||
// mask any cell yet.
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
||||
|
||||
|
@ -43,7 +44,7 @@ public class MinorCompactionScanQueryMatcher extends CompactionScanQueryMatcher
|
|||
}
|
||||
long mvccVersion = cell.getSequenceId();
|
||||
byte typeByte = cell.getTypeByte();
|
||||
if (CellUtil.isDelete(typeByte)) {
|
||||
if (PrivateCellUtil.isDelete(typeByte)) {
|
||||
if (mvccVersion > maxReadPointToTrackVersions) {
|
||||
// we should not use this delete marker to mask any cell yet.
|
||||
return MatchCode.INCLUDE;
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.TreeSet;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode;
|
||||
|
@ -165,7 +166,8 @@ public class NewVersionBehaviorTracker implements ColumnTracker, DeleteTracker {
|
|||
* Else return MAX_VALUE.
|
||||
*/
|
||||
protected long prepare(Cell cell) {
|
||||
boolean matchCq = CellUtil.matchingQualifier(cell, lastCqArray, lastCqOffset, lastCqLength);
|
||||
boolean matchCq =
|
||||
PrivateCellUtil.matchingQualifier(cell, lastCqArray, lastCqOffset, lastCqLength);
|
||||
if (!matchCq) {
|
||||
// The last cell is family-level delete and this is not, or the cq is changed,
|
||||
// we should construct delColMap as a deep copy of delFamMap.
|
||||
|
@ -175,7 +177,7 @@ public class NewVersionBehaviorTracker implements ColumnTracker, DeleteTracker {
|
|||
}
|
||||
countCurrentCol = 0;
|
||||
}
|
||||
if (matchCq && !CellUtil.isDelete(lastCqType) && lastCqType == cell.getTypeByte()
|
||||
if (matchCq && !PrivateCellUtil.isDelete(lastCqType) && lastCqType == cell.getTypeByte()
|
||||
&& lastCqTs == cell.getTimestamp()) {
|
||||
// Put with duplicate timestamp, ignore.
|
||||
return lastCqMvcc;
|
||||
|
@ -300,7 +302,7 @@ public class NewVersionBehaviorTracker implements ColumnTracker, DeleteTracker {
|
|||
@Override
|
||||
public MatchCode checkVersions(Cell cell, long timestamp, byte type,
|
||||
boolean ignoreCount) throws IOException {
|
||||
assert !CellUtil.isDelete(type);
|
||||
assert !PrivateCellUtil.isDelete(type);
|
||||
// We drop old version in #isDeleted, so here we won't SKIP because of versioning. But we should
|
||||
// consider TTL.
|
||||
if (ignoreCount) {
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -66,7 +67,7 @@ public abstract class NormalUserScanQueryMatcher extends UserScanQueryMatcher {
|
|||
}
|
||||
long timestamp = cell.getTimestamp();
|
||||
byte typeByte = cell.getTypeByte();
|
||||
if (CellUtil.isDelete(typeByte)) {
|
||||
if (PrivateCellUtil.isDelete(typeByte)) {
|
||||
boolean includeDeleteMarker = seePastDeleteMarkers ? tr.withinTimeRange(timestamp)
|
||||
: tr.withinOrAfterTimeRange(timestamp);
|
||||
if (includeDeleteMarker) {
|
||||
|
|
|
@ -25,6 +25,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.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
|
@ -148,7 +149,7 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
// Look for a TTL tag first. Use it instead of the family setting if
|
||||
// found. If a cell has multiple TTLs, resolve the conflict by using the
|
||||
// first tag encountered.
|
||||
Iterator<Tag> i = CellUtil.tagsIterator(cell);
|
||||
Iterator<Tag> i = PrivateCellUtil.tagsIterator(cell);
|
||||
while (i.hasNext()) {
|
||||
Tag t = i.next();
|
||||
if (TagType.TTL_TAG_TYPE == t.getType()) {
|
||||
|
@ -296,7 +297,7 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
// see TestFromClientSide3#testScanAfterDeletingSpecifiedRow
|
||||
// see TestFromClientSide3#testScanAfterDeletingSpecifiedRowV2
|
||||
if (cell.getQualifierLength() == 0) {
|
||||
Cell nextKey = CellUtil.createNextOnRowCol(cell);
|
||||
Cell nextKey = PrivateCellUtil.createNextOnRowCol(cell);
|
||||
if (nextKey != cell) {
|
||||
return nextKey;
|
||||
}
|
||||
|
@ -305,10 +306,10 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
}
|
||||
ColumnCount nextColumn = columns.getColumnHint();
|
||||
if (nextColumn == null) {
|
||||
return CellUtil.createLastOnRowCol(cell);
|
||||
return PrivateCellUtil.createLastOnRowCol(cell);
|
||||
} else {
|
||||
return CellUtil.createFirstOnRowCol(cell, nextColumn.getBuffer(), nextColumn.getOffset(),
|
||||
nextColumn.getLength());
|
||||
return PrivateCellUtil.createFirstOnRowCol(cell, nextColumn.getBuffer(),
|
||||
nextColumn.getOffset(), nextColumn.getLength());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -318,7 +319,7 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
* @return result of the compare between the indexed key and the key portion of the passed cell
|
||||
*/
|
||||
public int compareKeyForNextRow(Cell nextIndexed, Cell currentCell) {
|
||||
return CellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 0, 0, null, 0,
|
||||
return PrivateCellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 0, 0, null, 0,
|
||||
0, HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode());
|
||||
}
|
||||
|
||||
|
@ -330,10 +331,10 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
public int compareKeyForNextColumn(Cell nextIndexed, Cell currentCell) {
|
||||
ColumnCount nextColumn = columns.getColumnHint();
|
||||
if (nextColumn == null) {
|
||||
return CellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 0, 0, null,
|
||||
return PrivateCellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell, 0, 0, null,
|
||||
0, 0, HConstants.OLDEST_TIMESTAMP, Type.Minimum.getCode());
|
||||
} else {
|
||||
return CellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell,
|
||||
return PrivateCellUtil.compareKeyBasedOnColHint(rowComparator, nextIndexed, currentCell,
|
||||
currentCell.getFamilyOffset(), currentCell.getFamilyLength(), nextColumn.getBuffer(),
|
||||
nextColumn.getOffset(), nextColumn.getLength(), HConstants.LATEST_TIMESTAMP,
|
||||
Type.Maximum.getCode());
|
||||
|
@ -353,7 +354,7 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
@Override
|
||||
public void beforeShipped() throws IOException {
|
||||
if (this.currentRow != null) {
|
||||
this.currentRow = CellUtil.createFirstOnRow(CellUtil.copyRow(this.currentRow));
|
||||
this.currentRow = PrivateCellUtil.createFirstOnRow(CellUtil.copyRow(this.currentRow));
|
||||
}
|
||||
if (columns != null) {
|
||||
columns.beforeShipped();
|
||||
|
@ -361,7 +362,7 @@ public abstract class ScanQueryMatcher implements ShipperListener {
|
|||
}
|
||||
|
||||
protected static Cell createStartKeyFromRow(byte[] startRow, ScanInfo scanInfo) {
|
||||
return CellUtil.createFirstDeleteFamilyCellOnRow(startRow, scanInfo.getFamily());
|
||||
return PrivateCellUtil.createFirstDeleteFamilyCellOnRow(startRow, scanInfo.getFamily());
|
||||
}
|
||||
|
||||
protected static Pair<DeleteTracker, ColumnTracker> getTrackers(RegionCoprocessorHost host,
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode;
|
||||
|
@ -130,7 +131,7 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
|
|||
* delete
|
||||
*/
|
||||
private MatchCode checkVersion(byte type, long timestamp) {
|
||||
if (!CellUtil.isDelete(type)) {
|
||||
if (!PrivateCellUtil.isDelete(type)) {
|
||||
currentCount++;
|
||||
}
|
||||
if (currentCount > maxVersions) {
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
||||
|
||||
|
@ -56,7 +57,7 @@ public class StripeCompactionScanQueryMatcher extends DropDeletesCompactionScanQ
|
|||
}
|
||||
long mvccVersion = cell.getSequenceId();
|
||||
byte typeByte = cell.getTypeByte();
|
||||
if (CellUtil.isDelete(typeByte)) {
|
||||
if (PrivateCellUtil.isDelete(typeByte)) {
|
||||
if (mvccVersion > maxReadPointToTrackVersions) {
|
||||
return MatchCode.INCLUDE;
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.NavigableSet;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -63,7 +64,7 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
|
|||
if (scan.includeStartRow()) {
|
||||
return createStartKeyFromRow(scan.getStartRow(), scanInfo);
|
||||
} else {
|
||||
return CellUtil.createLastOnRow(scan.getStartRow());
|
||||
return PrivateCellUtil.createLastOnRow(scan.getStartRow());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -53,8 +53,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.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
|
@ -936,7 +936,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
|
|||
long len = 0;
|
||||
if (!listeners.isEmpty()) {
|
||||
for (Cell cell : e.getEdit().getCells()) {
|
||||
len += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
len += PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
}
|
||||
for (WALActionsListener listener : listeners) {
|
||||
listener.postAppend(len, elapsedTime, e.getKey(), e.getEdit());
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.TreeSet;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -116,7 +117,7 @@ class FSWALEntry extends Entry {
|
|||
long regionSequenceId = we.getWriteNumber();
|
||||
if (!this.getEdit().isReplay() && inMemstore) {
|
||||
for (Cell c : getEdit().getCells()) {
|
||||
CellUtil.setSequenceId(c, regionSequenceId);
|
||||
PrivateCellUtil.setSequenceId(c, regionSequenceId);
|
||||
}
|
||||
}
|
||||
getKey().setWriteEntry(we);
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue