HBASE-19628 ByteBufferCell should extend ExtendedCell
This commit is contained in:
parent
467a4667d8
commit
6b39062e86
|
@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
|
|||
import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
|
@ -78,7 +78,8 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool {
|
|||
@Override
|
||||
public void map(NullWritable key, Cell value, Context context)
|
||||
throws IOException, InterruptedException {
|
||||
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), new MapReduceCell(value));
|
||||
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)),
|
||||
new MapReduceExtendedCell(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -113,7 +114,7 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool {
|
|||
job.setReducerClass(CellSortReducer.class);
|
||||
Path outputDir = new Path(hfileOutPath);
|
||||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
job.setMapOutputValueClass(MapReduceCell.class);
|
||||
job.setMapOutputValueClass(MapReduceExtendedCell.class);
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table table = conn.getTable(tableName);
|
||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||
|
|
|
@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -95,9 +95,9 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
private static int compareQualifierPart(Cell cell, int length, byte[] prefix) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), length, prefix, 0, length);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), length, prefix, 0, length);
|
||||
}
|
||||
return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0,
|
||||
length);
|
||||
|
|
|
@ -18,21 +18,24 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
|
||||
/**
|
||||
* A filter that will only return the key component of each KV (the value will
|
||||
|
@ -60,8 +63,8 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
|
||||
private Cell createKeyOnlyCell(Cell c) {
|
||||
if (c instanceof ByteBufferCell) {
|
||||
return new KeyOnlyByteBufferCell((ByteBufferCell) c, lenAsVal);
|
||||
if (c instanceof ByteBufferExtendedCell) {
|
||||
return new KeyOnlyByteBufferExtendedCell((ByteBufferExtendedCell) c, lenAsVal);
|
||||
} else {
|
||||
return new KeyOnlyCell(c, lenAsVal);
|
||||
}
|
||||
|
@ -77,7 +80,7 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
public ReturnCode filterCell(final Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument((filterArguments.isEmpty() || filterArguments.size() == 1),
|
||||
"Expected: 0 or 1 but got: %s", filterArguments.size());
|
||||
|
@ -244,11 +247,13 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
static class KeyOnlyByteBufferCell extends ByteBufferCell {
|
||||
private ByteBufferCell cell;
|
||||
static class KeyOnlyByteBufferExtendedCell extends ByteBufferExtendedCell {
|
||||
public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
|
||||
+ Bytes.SIZEOF_BOOLEAN;
|
||||
private ByteBufferExtendedCell cell;
|
||||
private boolean lenAsVal;
|
||||
|
||||
public KeyOnlyByteBufferCell(ByteBufferCell c, boolean lenAsVal) {
|
||||
public KeyOnlyByteBufferExtendedCell(ByteBufferExtendedCell c, boolean lenAsVal) {
|
||||
this.cell = c;
|
||||
this.lenAsVal = lenAsVal;
|
||||
}
|
||||
|
@ -308,6 +313,21 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
return cell.getTypeByte();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSequenceId(long seqId) throws IOException {
|
||||
cell.setSequenceId(seqId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimestamp(long ts) throws IOException {
|
||||
cell.setTimestamp(ts);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimestamp(byte[] ts) throws IOException {
|
||||
cell.setTimestamp(ts);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSequenceId() {
|
||||
return 0;
|
||||
|
@ -409,6 +429,21 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
public int getTagsPosition() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Tag> getTags() {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSize() {
|
||||
return ClassSize.align(FIXED_OVERHEAD + cell.heapSize());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
|
@ -61,9 +61,9 @@ public class PrefixFilter extends FilterBase {
|
|||
// else return true, filter row
|
||||
// if we are passed the prefix, set flag
|
||||
int cmp;
|
||||
if (firstRowCell instanceof ByteBufferCell) {
|
||||
cmp = ByteBufferUtils.compareTo(((ByteBufferCell) firstRowCell).getRowByteBuffer(),
|
||||
((ByteBufferCell) firstRowCell).getRowPosition(), this.prefix.length,
|
||||
if (firstRowCell instanceof ByteBufferExtendedCell) {
|
||||
cmp = ByteBufferUtils.compareTo(((ByteBufferExtendedCell) firstRowCell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) firstRowCell).getRowPosition(), this.prefix.length,
|
||||
this.prefix, 0, this.prefix.length);
|
||||
} else {
|
||||
cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),
|
||||
|
|
|
@ -40,7 +40,7 @@ import java.util.stream.Collectors;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -2039,17 +2039,17 @@ public final class ProtobufUtil {
|
|||
// Doing this is going to kill us if we do it for all data passed.
|
||||
// St.Ack 20121205
|
||||
CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder();
|
||||
if (kv instanceof ByteBufferCell) {
|
||||
kvbuilder.setRow(wrap(((ByteBufferCell) kv).getRowByteBuffer(),
|
||||
((ByteBufferCell) kv).getRowPosition(), kv.getRowLength()));
|
||||
kvbuilder.setFamily(wrap(((ByteBufferCell) kv).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) kv).getFamilyPosition(), kv.getFamilyLength()));
|
||||
kvbuilder.setQualifier(wrap(((ByteBufferCell) kv).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) kv).getQualifierPosition(), kv.getQualifierLength()));
|
||||
if (kv instanceof ByteBufferExtendedCell) {
|
||||
kvbuilder.setRow(wrap(((ByteBufferExtendedCell) kv).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) kv).getRowPosition(), kv.getRowLength()));
|
||||
kvbuilder.setFamily(wrap(((ByteBufferExtendedCell) kv).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) kv).getFamilyPosition(), kv.getFamilyLength()));
|
||||
kvbuilder.setQualifier(wrap(((ByteBufferExtendedCell) kv).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) kv).getQualifierPosition(), kv.getQualifierLength()));
|
||||
kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
|
||||
kvbuilder.setTimestamp(kv.getTimestamp());
|
||||
kvbuilder.setValue(wrap(((ByteBufferCell) kv).getValueByteBuffer(),
|
||||
((ByteBufferCell) kv).getValuePosition(), kv.getValueLength()));
|
||||
kvbuilder.setValue(wrap(((ByteBufferExtendedCell) kv).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) kv).getValuePosition(), kv.getValueLength()));
|
||||
// TODO : Once tags become first class then we may have to set tags to kvbuilder.
|
||||
} else {
|
||||
kvbuilder.setRow(
|
||||
|
|
|
@ -20,9 +20,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -30,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
|
||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -82,7 +80,7 @@ public class TestKeyOnlyFilter {
|
|||
KeyValue KeyOnlyKeyValue = new KeyValue(newBuffer);
|
||||
|
||||
KeyOnlyCell keyOnlyCell = new KeyOnlyCell(kv, lenAsVal);
|
||||
KeyOnlyByteBufferCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferCell(
|
||||
KeyOnlyByteBufferExtendedCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferExtendedCell(
|
||||
bbCell, lenAsVal);
|
||||
|
||||
assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyCell));
|
||||
|
|
|
@ -45,29 +45,30 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
* when it is not.
|
||||
*/
|
||||
/*
|
||||
* Even though all the methods are abstract, ByteBufferCell is not made to be an interface with
|
||||
* intent. In CellComparator compare method, we have instance of check to decide whether to use
|
||||
* getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
|
||||
* if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
* Even though all the methods are abstract, ByteBufferExtendedCell is not made to be an interface
|
||||
* with intent. In CellComparator compare method, we have instance of check to decide whether to
|
||||
* use getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
|
||||
* if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
....
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
....
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
....
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
* We did JMH micro benchmark tests with both left and right cells as ByteBufferCell, one only
|
||||
* ByteBufferCell and both as Cells. This is compared against JMH results on compare logic with out
|
||||
* any instance of checks. We noticed that if ByteBufferCell is an interface, the benchmark result
|
||||
* seems to be very bad for case of both right and left are Cell only (Not ByteBufferCell). When
|
||||
* ByteBufferCell is an abstract class all 4 possible cases giving almost similar performance number
|
||||
* compared with compare logic with no instance of checks.
|
||||
* We did JMH micro benchmark tests with both left and right cells as ByteBufferExtendedCell, one
|
||||
* only ByteBufferExtendedCell and both as Cells. This is compared against JMH results on compare
|
||||
* logic with out any instance of checks. We noticed that if ByteBufferExtendedCell is an
|
||||
* interface, the benchmark result seems to be very bad for case of both right and left are Cell
|
||||
* only (Not ByteBufferExtendedCell). When ByteBufferExtendedCell is an abstract class all 4
|
||||
* possible cases giving almost similar performance number compared with compare logic with no
|
||||
* instance of checks.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ByteBufferCell implements Cell {
|
||||
public abstract class ByteBufferExtendedCell implements ExtendedCell {
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the row bytes.
|
||||
*/
|
|
@ -18,10 +18,14 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -30,8 +34,9 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
* (onheap and offheap).
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
|
||||
|
||||
public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
|
||||
public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
|
||||
+ (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT;
|
||||
private ByteBuffer buf;
|
||||
private int offset = 0; // offset into buffer where key starts at
|
||||
private int length = 0; // length of this.
|
||||
|
@ -151,6 +156,21 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
|
|||
return PrivateCellUtil.toDataType(getTypeByte());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSequenceId(long seqId) throws IOException {
|
||||
throw new IllegalArgumentException("This is a key only Cell");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimestamp(long ts) throws IOException {
|
||||
throw new IllegalArgumentException("This is a key only Cell");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimestamp(byte[] ts) throws IOException {
|
||||
throw new IllegalArgumentException("This is a key only Cell");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSequenceId() {
|
||||
return 0;
|
||||
|
@ -246,4 +266,22 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
|
|||
public String toString() {
|
||||
return CellUtil.toString(this, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Tag> getTags() {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long heapSize() {
|
||||
if (this.buf.hasArray()) {
|
||||
return ClassSize.align(FIXED_OVERHEAD + length);
|
||||
}
|
||||
return ClassSize.align(FIXED_OVERHEAD);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,11 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* This Cell is an implementation of {@link ByteBufferCell} where the data resides in
|
||||
* This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
|
||||
* off heap/ on heap ByteBuffer
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
|
||||
public class ByteBufferKeyValue extends ByteBufferExtendedCell {
|
||||
|
||||
protected final ByteBuffer buf;
|
||||
protected final int offset;
|
||||
|
@ -329,7 +329,7 @@ public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
|
|||
return calculateHashForKey(this);
|
||||
}
|
||||
|
||||
private int calculateHashForKey(ByteBufferCell cell) {
|
||||
private int calculateHashForKey(ByteBufferExtendedCell cell) {
|
||||
int rowHash = ByteBufferUtils.hashCode(cell.getRowByteBuffer(), cell.getRowPosition(),
|
||||
cell.getRowLength());
|
||||
int familyHash = ByteBufferUtils.hashCode(cell.getFamilyByteBuffer(), cell.getFamilyPosition(),
|
||||
|
|
|
@ -108,26 +108,26 @@ public class CellComparatorImpl implements CellComparator {
|
|||
*/
|
||||
@Override
|
||||
public final int compareFamilies(Cell left, Cell right) {
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
// Notice how we flip the order of the compare here. We used to negate the return value but
|
||||
// see what FindBugs says
|
||||
// http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
|
||||
// It suggest flipping the order to get same effect and 'safer'.
|
||||
return ByteBufferUtils.compareTo(
|
||||
left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
((ByteBufferCell)right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell)right).getFamilyPosition(), right.getFamilyLength());
|
||||
((ByteBufferExtendedCell)right).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell)right).getFamilyPosition(), right.getFamilyLength());
|
||||
}
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
|
@ -141,28 +141,28 @@ public class CellComparatorImpl implements CellComparator {
|
|||
*/
|
||||
@Override
|
||||
public final int compareQualifiers(Cell left, Cell right) {
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils
|
||||
.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(),
|
||||
left.getQualifierLength(), ((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(),
|
||||
.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(),
|
||||
left.getQualifierLength(), ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getQualifierPosition(),
|
||||
right.getQualifierLength());
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
// Notice how we flip the order of the compare here. We used to negate the return value but
|
||||
// see what FindBugs says
|
||||
// http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
|
||||
// It suggest flipping the order to get same effect and 'safer'.
|
||||
return ByteBufferUtils.compareTo(left.getQualifierArray(),
|
||||
left.getQualifierOffset(), left.getQualifierLength(),
|
||||
((ByteBufferCell)right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell)right).getQualifierPosition(), right.getQualifierLength());
|
||||
((ByteBufferExtendedCell)right).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell)right).getQualifierPosition(), right.getQualifierLength());
|
||||
}
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
|
||||
|
@ -183,25 +183,25 @@ public class CellComparatorImpl implements CellComparator {
|
|||
if (left == right) {
|
||||
return 0;
|
||||
}
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
|
||||
((ByteBufferCell) right).getRowByteBuffer(),
|
||||
((ByteBufferCell) right).getRowPosition(), right.getRowLength());
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
|
||||
((ByteBufferExtendedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength());
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
// Notice how we flip the order of the compare here. We used to negate the return value but
|
||||
// see what FindBugs says
|
||||
// http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
|
||||
// It suggest flipping the order to get same effect and 'safer'.
|
||||
return ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
((ByteBufferCell)right).getRowByteBuffer(),
|
||||
((ByteBufferCell)right).getRowPosition(), right.getRowLength());
|
||||
((ByteBufferExtendedCell)right).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell)right).getRowPosition(), right.getRowLength());
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
|
@ -225,9 +225,9 @@ public class CellComparatorImpl implements CellComparator {
|
|||
*/
|
||||
@Override
|
||||
public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(), right,
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
|
||||
|
|
|
@ -206,9 +206,10 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
short rowLen = cell.getRowLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
} else {
|
||||
System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
|
||||
rowLen);
|
||||
|
@ -225,9 +226,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyRowTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
short rowLen = cell.getRowLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
destination, ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
|
||||
cell.getRowOffset(), rowLen);
|
||||
|
@ -241,10 +242,10 @@ public final class CellUtil {
|
|||
* @return the byte[] containing the row
|
||||
*/
|
||||
public static byte[] copyRow(Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.copyOfRange(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(),
|
||||
((ByteBufferCell) cell).getRowPosition() + cell.getRowLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.copyOfRange(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition() + cell.getRowLength());
|
||||
} else {
|
||||
return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowOffset() + cell.getRowLength());
|
||||
|
@ -260,10 +261,10 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
byte fLen = cell.getFamilyLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(), ((ByteBufferCell) cell).getFamilyPosition(),
|
||||
destinationOffset, fLen);
|
||||
((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
|
||||
} else {
|
||||
System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
|
||||
destinationOffset, fLen);
|
||||
|
@ -280,9 +281,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyFamilyTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
byte fLen = cell.getFamilyLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getFamilyPosition(), destinationOffset, fLen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
destination, ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
|
||||
cell.getFamilyOffset(), fLen);
|
||||
|
@ -299,10 +300,10 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
int qlen = cell.getQualifierLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), destinationOffset, qlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
|
||||
destinationOffset, qlen);
|
||||
|
@ -319,9 +320,11 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyQualifierTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
int qlen = cell.getQualifierLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
destination, ((ByteBufferExtendedCell) cell).getQualifierPosition(),
|
||||
destinationOffset, qlen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
|
||||
|
@ -338,10 +341,10 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
int vlen = cell.getValueLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferCell) cell).getValueByteBuffer(), ((ByteBufferCell) cell).getValuePosition(),
|
||||
destinationOffset, vlen);
|
||||
((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
|
||||
vlen);
|
||||
|
@ -358,9 +361,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyValueTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
int vlen = cell.getValueLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getValuePosition(), destinationOffset, vlen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
destination, ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
|
||||
cell.getValueOffset(), vlen);
|
||||
|
@ -747,20 +750,20 @@ public final class CellUtil {
|
|||
public static boolean matchingFamily(final Cell left, final Cell right) {
|
||||
byte lfamlength = left.getFamilyLength();
|
||||
byte rfamlength = right.getFamilyLength();
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), lfamlength,
|
||||
((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), rfamlength);
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
|
||||
((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), lfamlength,
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
|
||||
right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), rfamlength,
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength,
|
||||
left.getFamilyArray(), left.getFamilyOffset(), lfamlength);
|
||||
}
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
|
||||
|
@ -786,20 +789,20 @@ public final class CellUtil {
|
|||
public static boolean matchingQualifier(final Cell left, final Cell right) {
|
||||
int lqlength = left.getQualifierLength();
|
||||
int rqlength = right.getQualifierLength();
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), lqlength,
|
||||
((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(), rqlength);
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
|
||||
((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), lqlength,
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
|
||||
right.getQualifierArray(), right.getQualifierOffset(), rqlength);
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(), rqlength,
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength,
|
||||
left.getQualifierArray(), left.getQualifierOffset(), lqlength);
|
||||
}
|
||||
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
|
@ -864,20 +867,20 @@ public final class CellUtil {
|
|||
|
||||
public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
|
||||
int rvlength) {
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), lvlength,
|
||||
((ByteBufferCell) right).getValueByteBuffer(),
|
||||
((ByteBufferCell) right).getValuePosition(), rvlength);
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
|
||||
((ByteBufferExtendedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), lvlength, right.getValueArray(),
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
|
||||
right.getValueOffset(), rvlength);
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getValueByteBuffer(),
|
||||
((ByteBufferCell) right).getValuePosition(), rvlength, left.getValueArray(),
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
|
||||
left.getValueOffset(), lvlength);
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
|
||||
|
@ -885,9 +888,9 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
public static boolean matchingValue(final Cell left, final byte[] buf) {
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), left.getValueLength(), buf, 0,
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
|
||||
buf.length) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
|
||||
|
@ -1199,9 +1202,11 @@ public final class CellUtil {
|
|||
@Deprecated
|
||||
public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell,
|
||||
int qlength, int commonPrefix) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream((DataOutput)out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyBufferToStream((DataOutput)out,
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition() + commonPrefix,
|
||||
qlength - commonPrefix);
|
||||
} else {
|
||||
out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
|
||||
qlength - commonPrefix);
|
||||
|
@ -1355,24 +1360,24 @@ public final class CellUtil {
|
|||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
if (lrowlength != rrowlength) return false;
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), lrowlength,
|
||||
((ByteBufferCell) right).getRowByteBuffer(), ((ByteBufferCell) right).getRowPosition(),
|
||||
rrowlength);
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
|
||||
((ByteBufferExtendedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), lrowlength, right.getRowArray(),
|
||||
right.getRowOffset(), rrowlength);
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
|
||||
right.getRowOffset(), rrowlength);
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getRowByteBuffer(),
|
||||
((ByteBufferCell) right).getRowPosition(), rrowlength, left.getRowArray(),
|
||||
left.getRowOffset(), lrowlength);
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
|
||||
left.getRowOffset(), lrowlength);
|
||||
}
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
|
||||
right.getRowOffset(), rrowlength);
|
||||
right.getRowOffset(), rrowlength);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1433,10 +1438,10 @@ public final class CellUtil {
|
|||
* cell's qualifier is lesser than byte[] and 0 otherwise
|
||||
*/
|
||||
public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(), right, rOffset,
|
||||
rLength);
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(),
|
||||
left.getQualifierLength(), right, rOffset, rLength);
|
||||
}
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right, rOffset, rLength);
|
||||
|
@ -1481,9 +1486,9 @@ public final class CellUtil {
|
|||
* cell's family is lesser than byte[] and 0 otherwise
|
||||
*/
|
||||
public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
|
||||
rlength);
|
||||
}
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
* {@link Tag} and memstoreTS/mvcc are internal implementation detail
|
||||
* that should not be exposed publicly.
|
||||
* Use {@link ExtendedCellBuilderFactory} to get ExtendedCellBuilder instance.
|
||||
* TODO: ditto for ByteBufferCell?
|
||||
* TODO: ditto for ByteBufferExtendedCell?
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ExtendedCellBuilder extends RawCellBuilder {
|
||||
|
|
|
@ -83,17 +83,17 @@ public final class PrivateCellUtil {
|
|||
/********************* misc *************************************/
|
||||
|
||||
public static byte getRowByte(Cell cell, int index) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) cell).getRowByteBuffer()
|
||||
.get(((ByteBufferCell) cell).getRowPosition() + index);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) cell).getRowByteBuffer()
|
||||
.get(((ByteBufferExtendedCell) cell).getRowPosition() + index);
|
||||
}
|
||||
return cell.getRowArray()[cell.getRowOffset() + index];
|
||||
}
|
||||
|
||||
public static byte getQualifierByte(Cell cell, int index) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) cell).getQualifierByteBuffer()
|
||||
.get(((ByteBufferCell) cell).getQualifierPosition() + index);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) cell).getQualifierByteBuffer()
|
||||
.get(((ByteBufferExtendedCell) cell).getQualifierPosition() + index);
|
||||
}
|
||||
return cell.getQualifierArray()[cell.getQualifierOffset() + index];
|
||||
}
|
||||
|
@ -115,15 +115,16 @@ public final class PrivateCellUtil {
|
|||
* @return A new cell which is having the extra tags also added to it.
|
||||
*/
|
||||
public static Cell createCell(Cell cell, byte[] tags) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new TagRewriteByteBufferCell((ByteBufferCell) cell, tags);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell, tags);
|
||||
}
|
||||
return new TagRewriteCell(cell, tags);
|
||||
}
|
||||
|
||||
public static Cell createCell(Cell cell, byte[] value, byte[] tags) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) cell, value, tags);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new ValueAndTagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell,
|
||||
value, tags);
|
||||
}
|
||||
return new ValueAndTagRewriteCell(cell, value, tags);
|
||||
}
|
||||
|
@ -314,25 +315,24 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
static class TagRewriteByteBufferCell extends ByteBufferCell implements ExtendedCell {
|
||||
static class TagRewriteByteBufferExtendedCell extends ByteBufferExtendedCell {
|
||||
|
||||
protected ByteBufferCell cell;
|
||||
protected ByteBufferExtendedCell cell;
|
||||
protected byte[] tags;
|
||||
private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
|
||||
|
||||
/**
|
||||
* @param cell The original ByteBufferCell which it rewrites
|
||||
* @param cell The original ByteBufferExtendedCell which it rewrites
|
||||
* @param tags the tags bytes. The array suppose to contain the tags bytes alone.
|
||||
*/
|
||||
public TagRewriteByteBufferCell(ByteBufferCell cell, byte[] tags) {
|
||||
assert cell instanceof ExtendedCell;
|
||||
public TagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell, byte[] tags) {
|
||||
assert tags != null;
|
||||
this.cell = cell;
|
||||
this.tags = tags;
|
||||
// tag offset will be treated as 0 and length this.tags.length
|
||||
if (this.cell instanceof TagRewriteByteBufferCell) {
|
||||
if (this.cell instanceof TagRewriteByteBufferExtendedCell) {
|
||||
// Cleaning the ref so that the byte[] can be GCed
|
||||
((TagRewriteByteBufferCell) this.cell).tags = null;
|
||||
((TagRewriteByteBufferExtendedCell) this.cell).tags = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -490,8 +490,9 @@ public final class PrivateCellUtil {
|
|||
@Override
|
||||
public ExtendedCell deepClone() {
|
||||
Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
|
||||
if (clonedBaseCell instanceof ByteBufferCell) {
|
||||
return new TagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, this.tags);
|
||||
if (clonedBaseCell instanceof ByteBufferExtendedCell) {
|
||||
return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) clonedBaseCell,
|
||||
this.tags);
|
||||
}
|
||||
return new TagRewriteCell(clonedBaseCell, this.tags);
|
||||
}
|
||||
|
@ -585,7 +586,10 @@ public final class PrivateCellUtil {
|
|||
return write(out, withTags, this.cell, this.value, this.tags);
|
||||
}
|
||||
|
||||
// Made into a static method so as to reuse the logic within ValueAndTagRewriteByteBufferCell
|
||||
/**
|
||||
* Made into a static method so as to reuse the logic within
|
||||
* ValueAndTagRewriteByteBufferExtendedCell
|
||||
*/
|
||||
static int write(OutputStream out, boolean withTags, Cell cell, byte[] value, byte[] tags)
|
||||
throws IOException {
|
||||
int valLen = value == null ? 0 : value.length;
|
||||
|
@ -617,7 +621,10 @@ public final class PrivateCellUtil {
|
|||
write(buf, offset, this.cell, this.value, this.tags);
|
||||
}
|
||||
|
||||
// Made into a static method so as to reuse the logic within ValueAndTagRewriteByteBufferCell
|
||||
/**
|
||||
* Made into a static method so as to reuse the logic
|
||||
* within ValueAndTagRewriteByteBufferExtendedCell
|
||||
*/
|
||||
static void write(ByteBuffer buf, int offset, Cell cell, byte[] value, byte[] tags) {
|
||||
offset = ByteBufferUtils.putInt(buf, offset, KeyValueUtil.keyLength(cell));// Key length
|
||||
offset = ByteBufferUtils.putInt(buf, offset, value.length);// Value length
|
||||
|
@ -638,11 +645,12 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
static class ValueAndTagRewriteByteBufferCell extends TagRewriteByteBufferCell {
|
||||
static class ValueAndTagRewriteByteBufferExtendedCell extends TagRewriteByteBufferExtendedCell {
|
||||
|
||||
protected byte[] value;
|
||||
|
||||
public ValueAndTagRewriteByteBufferCell(ByteBufferCell cell, byte[] value, byte[] tags) {
|
||||
public ValueAndTagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell,
|
||||
byte[] value, byte[] tags) {
|
||||
super(cell, tags);
|
||||
this.value = value;
|
||||
}
|
||||
|
@ -699,9 +707,9 @@ public final class PrivateCellUtil {
|
|||
@Override
|
||||
public ExtendedCell deepClone() {
|
||||
Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
|
||||
if (clonedBaseCell instanceof ByteBufferCell) {
|
||||
return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, this.value,
|
||||
this.tags);
|
||||
if (clonedBaseCell instanceof ByteBufferExtendedCell) {
|
||||
return new ValueAndTagRewriteByteBufferExtendedCell(
|
||||
(ByteBufferExtendedCell) clonedBaseCell, this.value, this.tags);
|
||||
}
|
||||
return new ValueAndTagRewriteCell(clonedBaseCell, this.value, this.tags);
|
||||
}
|
||||
|
@ -709,19 +717,21 @@ public final class PrivateCellUtil {
|
|||
|
||||
public static boolean matchingRows(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
|
||||
buf, offset, length);
|
||||
}
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
|
||||
length);
|
||||
length);
|
||||
}
|
||||
|
||||
public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset, length);
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
buf, offset, length);
|
||||
}
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
|
||||
offset, length);
|
||||
|
@ -740,10 +750,10 @@ public final class PrivateCellUtil {
|
|||
if (buf == null) {
|
||||
return left.getQualifierLength() == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(), buf, offset,
|
||||
length);
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
buf, offset, length);
|
||||
}
|
||||
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), buf, offset, length);
|
||||
|
@ -759,24 +769,25 @@ public final class PrivateCellUtil {
|
|||
|
||||
public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
|
||||
int rvlength) {
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), lvlength,
|
||||
((ByteBufferCell) right).getValueByteBuffer(), ((ByteBufferCell) right).getValuePosition(),
|
||||
rvlength);
|
||||
if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
|
||||
((ByteBufferExtendedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), lvlength, right.getValueArray(),
|
||||
right.getValueOffset(), rvlength);
|
||||
if (left instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
|
||||
right.getValueOffset(), rvlength);
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getValueByteBuffer(),
|
||||
((ByteBufferCell) right).getValuePosition(), rvlength, left.getValueArray(),
|
||||
left.getValueOffset(), lvlength);
|
||||
if (right instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
|
||||
left.getValueOffset(), lvlength);
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
|
||||
right.getValueArray(), right.getValueOffset(), rvlength);
|
||||
return Bytes
|
||||
.equals(left.getValueArray(), left.getValueOffset(), lvlength, right.getValueArray(),
|
||||
right.getValueOffset(), rvlength);
|
||||
}
|
||||
|
||||
public static boolean matchingType(Cell a, Cell b) {
|
||||
|
@ -837,13 +848,13 @@ public final class PrivateCellUtil {
|
|||
*/
|
||||
public static int copyTagsTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
int tlen = cell.getTagsLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferCell) cell).getTagsByteBuffer(), ((ByteBufferCell) cell).getTagsPosition(),
|
||||
destinationOffset, tlen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils
|
||||
.copyFromBufferToArray(destination, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
|
||||
tlen);
|
||||
System
|
||||
.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset, tlen);
|
||||
}
|
||||
return destinationOffset + tlen;
|
||||
}
|
||||
|
@ -857,9 +868,9 @@ public final class PrivateCellUtil {
|
|||
*/
|
||||
public static int copyTagsTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
int tlen = cell.getTagsLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
|
||||
destination, ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
|
||||
cell.getTagsOffset(), tlen);
|
||||
|
@ -887,14 +898,15 @@ public final class PrivateCellUtil {
|
|||
* @return null if there is no tag of the passed in tag type
|
||||
*/
|
||||
public static Optional<Tag> getTag(Cell cell, byte type) {
|
||||
boolean bufferBacked = cell instanceof ByteBufferCell;
|
||||
boolean bufferBacked = cell instanceof ByteBufferExtendedCell;
|
||||
int length = cell.getTagsLength();
|
||||
int offset = bufferBacked ? ((ByteBufferCell) cell).getTagsPosition() : cell.getTagsOffset();
|
||||
int offset =
|
||||
bufferBacked ? ((ByteBufferExtendedCell) cell).getTagsPosition() : cell.getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen;
|
||||
if (bufferBacked) {
|
||||
ByteBuffer tagsBuffer = ((ByteBufferCell) cell).getTagsByteBuffer();
|
||||
ByteBuffer tagsBuffer = ((ByteBufferExtendedCell) cell).getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return Optional.of(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
|
@ -922,9 +934,9 @@ public final class PrivateCellUtil {
|
|||
if (tagsLength == 0) {
|
||||
return TagUtil.EMPTY_TAGS_ITR;
|
||||
}
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), tagsLength);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return tagsIterator(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
|
||||
}
|
||||
return CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
|
||||
}
|
||||
|
@ -1000,10 +1012,11 @@ public final class PrivateCellUtil {
|
|||
*/
|
||||
public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
|
||||
int commonPrefix) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream((DataOutput) out,
|
||||
((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils
|
||||
.copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix,
|
||||
rlength - commonPrefix);
|
||||
} else {
|
||||
out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
|
||||
}
|
||||
|
@ -1028,23 +1041,27 @@ public final class PrivateCellUtil {
|
|||
int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
|
||||
if (rLen1 != rLen2) {
|
||||
// early out when the RK length itself is not matching
|
||||
return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
|
||||
Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
|
||||
return ByteBufferUtils
|
||||
.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE, Bytes.toBytes(rLen2),
|
||||
0, KeyValue.ROW_LENGTH_SIZE);
|
||||
}
|
||||
// Compare the RKs
|
||||
int rkCommonPrefix = 0;
|
||||
if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
|
||||
rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getRowByteBuffer(),
|
||||
((ByteBufferCell) c1).getRowPosition(), rLen1, ((ByteBufferCell) c2).getRowByteBuffer(),
|
||||
((ByteBufferCell) c2).getRowPosition(), rLen2);
|
||||
if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
|
||||
rkCommonPrefix = ByteBufferUtils
|
||||
.findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
|
||||
((ByteBufferExtendedCell) c2).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
|
||||
} else {
|
||||
// There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
|
||||
// either
|
||||
// in flush or compactions. In flushes both cells are KV and in case of compaction it will be
|
||||
// either
|
||||
// KV or BBCell
|
||||
rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1,
|
||||
c2.getRowArray(), c2.getRowOffset(), rLen2);
|
||||
rkCommonPrefix = ByteBufferUtils
|
||||
.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1, c2.getRowArray(),
|
||||
c2.getRowOffset(), rLen2);
|
||||
}
|
||||
commonPrefix += rkCommonPrefix;
|
||||
if (rkCommonPrefix != rLen1) {
|
||||
|
@ -1067,14 +1084,16 @@ public final class PrivateCellUtil {
|
|||
commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
|
||||
// Compare the CF names
|
||||
int fCommonPrefix;
|
||||
if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
|
||||
fCommonPrefix = ByteBufferUtils.findCommonPrefix(
|
||||
((ByteBufferCell) c1).getFamilyByteBuffer(), ((ByteBufferCell) c1).getFamilyPosition(),
|
||||
fLen1, ((ByteBufferCell) c2).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) c2).getFamilyPosition(), fLen2);
|
||||
if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
|
||||
fCommonPrefix = ByteBufferUtils
|
||||
.findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
|
||||
((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
|
||||
} else {
|
||||
fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
|
||||
fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
|
||||
fCommonPrefix = ByteBufferUtils
|
||||
.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(), fLen1, c2.getFamilyArray(),
|
||||
c2.getFamilyOffset(), fLen2);
|
||||
}
|
||||
commonPrefix += fCommonPrefix;
|
||||
if (fCommonPrefix != fLen1) {
|
||||
|
@ -1085,22 +1104,25 @@ public final class PrivateCellUtil {
|
|||
int qLen1 = c1.getQualifierLength();
|
||||
int qLen2 = c2.getQualifierLength();
|
||||
int qCommon;
|
||||
if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
|
||||
qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) c1).getQualifierPosition(), qLen1,
|
||||
((ByteBufferCell) c2).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) c2).getQualifierPosition(), qLen2);
|
||||
if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
|
||||
qCommon = ByteBufferUtils
|
||||
.findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
|
||||
((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
|
||||
} else {
|
||||
qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
|
||||
qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
|
||||
qCommon = ByteBufferUtils
|
||||
.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(), qLen1,
|
||||
c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
|
||||
}
|
||||
commonPrefix += qCommon;
|
||||
if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
|
||||
return commonPrefix;
|
||||
}
|
||||
// Compare the timestamp parts
|
||||
int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
|
||||
KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
|
||||
int tsCommonPrefix = ByteBufferUtils
|
||||
.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE,
|
||||
Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
|
||||
commonPrefix += tsCommonPrefix;
|
||||
if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
|
||||
return commonPrefix;
|
||||
|
@ -1195,9 +1217,9 @@ public final class PrivateCellUtil {
|
|||
* @return result comparing cell's row
|
||||
*/
|
||||
public static int compareRow(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return comparator.compareTo(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
@ -1209,9 +1231,9 @@ public final class PrivateCellUtil {
|
|||
* @return result comparing cell's column family
|
||||
*/
|
||||
public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return comparator.compareTo(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength());
|
||||
|
@ -1224,9 +1246,9 @@ public final class PrivateCellUtil {
|
|||
* @return result comparing cell's qualifier
|
||||
*/
|
||||
public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return comparator.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
|
@ -1262,9 +1284,9 @@ public final class PrivateCellUtil {
|
|||
* @return result comparing cell's value
|
||||
*/
|
||||
public static int compareValue(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return comparator.compareTo(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
}
|
||||
|
@ -1375,8 +1397,7 @@ public final class PrivateCellUtil {
|
|||
* These cells are used in reseeks/seeks to improve the read performance. They are not real cells
|
||||
* that are returned back to the clients
|
||||
*/
|
||||
private static abstract class EmptyByteBufferCell extends ByteBufferCell
|
||||
implements ExtendedCell {
|
||||
private static abstract class EmptyByteBufferExtendedCell extends ByteBufferExtendedCell {
|
||||
|
||||
@Override
|
||||
public void setSequenceId(long seqId) {
|
||||
|
@ -1578,7 +1599,7 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
private static class FirstOnRowByteBufferCell extends EmptyByteBufferCell {
|
||||
private static class FirstOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
|
||||
private static final int FIXED_OVERHEAD =
|
||||
ClassSize.OBJECT // object
|
||||
+ ClassSize.REFERENCE // row buffer
|
||||
|
@ -1588,7 +1609,7 @@ public final class PrivateCellUtil {
|
|||
private final int roffset;
|
||||
private final short rlength;
|
||||
|
||||
public FirstOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
public FirstOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
this.rowBuff = row;
|
||||
this.roffset = roffset;
|
||||
this.rlength = rlength;
|
||||
|
@ -1633,7 +1654,7 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
private static class LastOnRowByteBufferCell extends EmptyByteBufferCell {
|
||||
private static class LastOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
|
||||
private static final int FIXED_OVERHEAD =
|
||||
ClassSize.OBJECT // object
|
||||
+ ClassSize.REFERENCE // rowBuff
|
||||
|
@ -1643,7 +1664,7 @@ public final class PrivateCellUtil {
|
|||
private final int roffset;
|
||||
private final short rlength;
|
||||
|
||||
public LastOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
public LastOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
this.rowBuff = row;
|
||||
this.roffset = roffset;
|
||||
this.rlength = rlength;
|
||||
|
@ -1688,9 +1709,10 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
private static class FirstOnRowColByteBufferCell extends FirstOnRowByteBufferCell {
|
||||
private static class FirstOnRowColByteBufferExtendedCell
|
||||
extends FirstOnRowByteBufferExtendedCell {
|
||||
private static final int FIXED_OVERHEAD =
|
||||
FirstOnRowByteBufferCell.FIXED_OVERHEAD
|
||||
FirstOnRowByteBufferExtendedCell.FIXED_OVERHEAD
|
||||
+ ClassSize.REFERENCE * 2 // family buffer and column buffer
|
||||
+ Bytes.SIZEOF_INT * 3 // famOffset, colOffset, colLength
|
||||
+ Bytes.SIZEOF_BYTE; // famLength
|
||||
|
@ -1701,7 +1723,7 @@ public final class PrivateCellUtil {
|
|||
private final int colOffset;
|
||||
private final int colLength;
|
||||
|
||||
public FirstOnRowColByteBufferCell(final ByteBuffer row, int roffset, short rlength,
|
||||
public FirstOnRowColByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength,
|
||||
final ByteBuffer famBuff, final int famOffset, final byte famLength, final ByteBuffer col,
|
||||
final int colOffset, final int colLength) {
|
||||
super(row, roffset, rlength);
|
||||
|
@ -1843,13 +1865,14 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
private static class FirstOnRowColTSByteBufferCell extends FirstOnRowColByteBufferCell {
|
||||
private static class FirstOnRowColTSByteBufferExtendedCell
|
||||
extends FirstOnRowColByteBufferExtendedCell {
|
||||
private static final int FIXED_OVERHEAD =
|
||||
FirstOnRowColByteBufferCell.FIXED_OVERHEAD
|
||||
FirstOnRowColByteBufferExtendedCell.FIXED_OVERHEAD
|
||||
+ Bytes.SIZEOF_LONG; // ts
|
||||
private long ts;
|
||||
|
||||
public FirstOnRowColTSByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
public FirstOnRowColTSByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset, int qlength,
|
||||
long ts) {
|
||||
super(rBuffer, roffset, rlength, fBuffer, foffset, flength, qBuffer, qoffset, qlength);
|
||||
|
@ -1983,9 +2006,9 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
}
|
||||
|
||||
private static class LastOnRowColByteBufferCell extends LastOnRowByteBufferCell {
|
||||
private static class LastOnRowColByteBufferExtendedCell extends LastOnRowByteBufferExtendedCell {
|
||||
private static final int FIXED_OVERHEAD =
|
||||
LastOnRowByteBufferCell.FIXED_OVERHEAD
|
||||
LastOnRowByteBufferExtendedCell.FIXED_OVERHEAD
|
||||
+ ClassSize.REFERENCE * 2 // fBuffer and qBuffer
|
||||
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
|
||||
+ Bytes.SIZEOF_BYTE; // flength
|
||||
|
@ -1996,7 +2019,7 @@ public final class PrivateCellUtil {
|
|||
private final int qoffset;
|
||||
private final int qlength;
|
||||
|
||||
public LastOnRowColByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
public LastOnRowColByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset,
|
||||
int qlength) {
|
||||
super(rBuffer, roffset, rlength);
|
||||
|
@ -2125,15 +2148,16 @@ public final class PrivateCellUtil {
|
|||
int qLen = cell.getQualifierLength();
|
||||
// Using just one if/else loop instead of every time checking before writing every
|
||||
// component of cell
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
out.writeShort(rowLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), rowLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
|
||||
out.writeByte(fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), qLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
|
||||
ByteBufferUtils
|
||||
.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
|
||||
} else {
|
||||
out.writeShort(rowLen);
|
||||
out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
|
||||
|
@ -2214,15 +2238,16 @@ public final class PrivateCellUtil {
|
|||
int qLen = cell.getQualifierLength();
|
||||
// Using just one if/else loop instead of every time checking before writing every
|
||||
// component of cell
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
StreamUtils.writeShort(out, rowLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), rowLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
|
||||
out.write(fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), qLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
|
||||
ByteBufferUtils
|
||||
.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
|
||||
} else {
|
||||
StreamUtils.writeShort(out, rowLen);
|
||||
out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
|
||||
|
@ -2233,7 +2258,7 @@ public final class PrivateCellUtil {
|
|||
StreamUtils.writeLong(out, cell.getTimestamp());
|
||||
out.write(cell.getTypeByte());
|
||||
return Bytes.SIZEOF_SHORT + rowLen + Bytes.SIZEOF_BYTE + fLen + qLen + Bytes.SIZEOF_LONG
|
||||
+ Bytes.SIZEOF_BYTE;
|
||||
+ Bytes.SIZEOF_BYTE;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2322,9 +2347,9 @@ public final class PrivateCellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeRow(OutputStream out, Cell cell, short rlength) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), rlength);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), rlength);
|
||||
} else {
|
||||
out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
|
||||
}
|
||||
|
@ -2338,9 +2363,9 @@ public final class PrivateCellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeFamily(OutputStream out, Cell cell, byte flength) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), flength);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), flength);
|
||||
} else {
|
||||
out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
|
||||
}
|
||||
|
@ -2354,9 +2379,10 @@ public final class PrivateCellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeQualifier(OutputStream out, Cell cell, int qlength) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), qlength);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils
|
||||
.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), qlength);
|
||||
} else {
|
||||
out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
|
||||
}
|
||||
|
@ -2371,13 +2397,14 @@ public final class PrivateCellUtil {
|
|||
*/
|
||||
public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell, int qlength,
|
||||
int commonPrefix) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyBufferToStream((DataOutput) out,
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition() + commonPrefix,
|
||||
qlength - commonPrefix);
|
||||
} else {
|
||||
out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
|
||||
qlength - commonPrefix);
|
||||
qlength - commonPrefix);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2389,9 +2416,9 @@ public final class PrivateCellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeValue(OutputStream out, Cell cell, int vlength) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), vlength);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition(), vlength);
|
||||
} else {
|
||||
out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
|
||||
}
|
||||
|
@ -2405,9 +2432,9 @@ public final class PrivateCellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeTags(OutputStream out, Cell cell, int tagsLength) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), tagsLength);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
|
||||
} else {
|
||||
out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
|
||||
}
|
||||
|
@ -2440,9 +2467,9 @@ public final class PrivateCellUtil {
|
|||
* @return rowkey as int
|
||||
*/
|
||||
public static int getRowAsInt(Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition());
|
||||
}
|
||||
return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
|
||||
}
|
||||
|
@ -2453,9 +2480,9 @@ public final class PrivateCellUtil {
|
|||
* @return value as long
|
||||
*/
|
||||
public static long getValueAsLong(Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toLong(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.toLong(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition());
|
||||
}
|
||||
return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
@ -2466,9 +2493,9 @@ public final class PrivateCellUtil {
|
|||
* @return value as int
|
||||
*/
|
||||
public static int getValueAsInt(Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition());
|
||||
}
|
||||
return Bytes.toInt(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
@ -2479,9 +2506,9 @@ public final class PrivateCellUtil {
|
|||
* @return value as double
|
||||
*/
|
||||
public static double getValueAsDouble(Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toDouble(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.toDouble(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition());
|
||||
}
|
||||
return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
@ -2492,9 +2519,9 @@ public final class PrivateCellUtil {
|
|||
* @return value as BigDecimal
|
||||
*/
|
||||
public static BigDecimal getValueAsBigDecimal(Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toBigDecimal(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.toBigDecimal(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
|
||||
}
|
||||
return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
}
|
||||
|
@ -2508,9 +2535,9 @@ public final class PrivateCellUtil {
|
|||
*/
|
||||
public static void compressTags(OutputStream out, Cell cell,
|
||||
TagCompressionContext tagCompressionContext) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
tagCompressionContext.compressTags(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), cell.getTagsLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
tagCompressionContext.compressTags(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getTagsPosition(), cell.getTagsLength());
|
||||
} else {
|
||||
tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
|
@ -2518,9 +2545,9 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
|
||||
public static void compressRow(OutputStream out, Cell cell, Dictionary dict) throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
Dictionary.write(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(), dict);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
Dictionary.write(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(), dict);
|
||||
} else {
|
||||
Dictionary.write(out, cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), dict);
|
||||
}
|
||||
|
@ -2528,9 +2555,9 @@ public final class PrivateCellUtil {
|
|||
|
||||
public static void compressFamily(OutputStream out, Cell cell, Dictionary dict)
|
||||
throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
Dictionary.write(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(), dict);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
Dictionary.write(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(), dict);
|
||||
} else {
|
||||
Dictionary.write(out, cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
dict);
|
||||
|
@ -2539,9 +2566,9 @@ public final class PrivateCellUtil {
|
|||
|
||||
public static void compressQualifier(OutputStream out, Cell cell, Dictionary dict)
|
||||
throws IOException {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
Dictionary.write(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength(), dict);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
Dictionary.write(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), dict);
|
||||
} else {
|
||||
Dictionary.write(out, cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), dict);
|
||||
|
@ -2668,13 +2695,14 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
|
||||
static Cell createNextOnRowCol(Cell cell, long ts, byte type) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new LastOnRowColByteBufferCell(
|
||||
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||
cell.getRowLength(), ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength()) {
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new LastOnRowColByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength()) {
|
||||
@Override
|
||||
public long getTimestamp() {
|
||||
return ts;
|
||||
|
@ -2686,8 +2714,8 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
};
|
||||
}
|
||||
return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()) {
|
||||
@Override
|
||||
public long getTimestamp() {
|
||||
|
@ -2788,13 +2816,12 @@ public final class PrivateCellUtil {
|
|||
* @return First possible Cell on passed Cell's row.
|
||||
*/
|
||||
public static Cell createFirstOnRow(final Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowByteBufferCell(
|
||||
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||
cell.getRowLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new FirstOnRowByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
|
||||
}
|
||||
return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength());
|
||||
return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
||||
public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength) {
|
||||
|
@ -2803,7 +2830,7 @@ public final class PrivateCellUtil {
|
|||
|
||||
public static Cell createFirstOnRow(final byte[] row, final byte[] family, final byte[] col) {
|
||||
return createFirstOnRow(row, 0, (short) row.length, family, 0, (byte) family.length, col, 0,
|
||||
col.length);
|
||||
col.length);
|
||||
}
|
||||
|
||||
public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength,
|
||||
|
@ -2817,23 +2844,24 @@ public final class PrivateCellUtil {
|
|||
}
|
||||
|
||||
public static Cell createFirstOnRowFamily(Cell cell, byte[] fArray, int foff, int flen) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColByteBufferCell(
|
||||
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||
cell.getRowLength(), ByteBuffer.wrap(fArray), foff, (byte) flen,
|
||||
HConstants.EMPTY_BYTE_BUFFER, 0, 0);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new FirstOnRowColByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
ByteBuffer.wrap(fArray), foff, (byte) flen, HConstants.EMPTY_BYTE_BUFFER, 0, 0);
|
||||
}
|
||||
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
fArray, foff, (byte) flen, HConstants.EMPTY_BYTE_ARRAY, 0, 0);
|
||||
}
|
||||
|
||||
public static Cell createFirstOnRowCol(final Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColByteBufferCell(
|
||||
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||
cell.getRowLength(), HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new FirstOnRowColByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
}
|
||||
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
HConstants.EMPTY_BYTE_ARRAY, 0, (byte) 0, cell.getQualifierArray(),
|
||||
|
@ -2857,16 +2885,17 @@ public final class PrivateCellUtil {
|
|||
* @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
|
||||
*/
|
||||
public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColByteBufferCell(
|
||||
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||
cell.getRowLength(), ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new FirstOnRowColByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
ByteBuffer.wrap(qArray), qoffest, qlength);
|
||||
}
|
||||
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
qArray, qoffest, qlength);
|
||||
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), qArray, qoffest,
|
||||
qlength);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2877,13 +2906,14 @@ public final class PrivateCellUtil {
|
|||
* @param ts
|
||||
*/
|
||||
public static Cell createFirstOnRowColTS(Cell cell, long ts) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColTSByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength(), ts);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new FirstOnRowColTSByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), ts);
|
||||
}
|
||||
return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
|
@ -2896,10 +2926,9 @@ public final class PrivateCellUtil {
|
|||
* @return Last possible Cell on passed Cell's row.
|
||||
*/
|
||||
public static Cell createLastOnRow(final Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new LastOnRowByteBufferCell(
|
||||
((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
|
||||
cell.getRowLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new LastOnRowByteBufferExtendedCell(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
|
||||
}
|
||||
return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
@ -2916,13 +2945,14 @@ public final class PrivateCellUtil {
|
|||
* @return Last possible Cell on passed Cell's rk:cf:q.
|
||||
*/
|
||||
public static Cell createLastOnRowCol(final Cell cell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new LastOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return new LastOnRowColByteBufferExtendedCell(
|
||||
((ByteBufferExtendedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
}
|
||||
return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
|
|
|
@ -99,9 +99,9 @@ public final class TagUtil {
|
|||
}
|
||||
byte[] b = new byte[tags.length + cellTagsLen];
|
||||
int pos = Bytes.putBytes(b, 0, tags, 0, tags.length);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(b, ((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), pos, cellTagsLen);
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(b, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferExtendedCell) cell).getTagsPosition(), pos, cellTagsLen);
|
||||
} else {
|
||||
Bytes.putBytes(b, pos, cell.getTagsArray(), cell.getTagsOffset(), cellTagsLen);
|
||||
}
|
||||
|
|
|
@ -21,8 +21,7 @@ import java.io.DataOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -252,7 +251,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
|
||||
private Cell toOffheapCell(ByteBuffer valAndTagsBuffer, int vOffset,
|
||||
int tagsLenSerializationSize) {
|
||||
ByteBuffer tagsBuf = HConstants.EMPTY_BYTE_BUFFER;
|
||||
ByteBuffer tagsBuf = HConstants.EMPTY_BYTE_BUFFER;
|
||||
int tOffset = 0;
|
||||
if (this.includeTags) {
|
||||
if (this.tagCompressionContext == null) {
|
||||
|
@ -263,8 +262,9 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
tOffset = 0;
|
||||
}
|
||||
}
|
||||
return new OffheapDecodedCell(ByteBuffer.wrap(Bytes.copy(keyBuffer, 0, this.keyLength)),
|
||||
currentKey.getRowLength(), currentKey.getFamilyOffset(), currentKey.getFamilyLength(),
|
||||
return new OffheapDecodedExtendedCell(
|
||||
ByteBuffer.wrap(Bytes.copy(keyBuffer, 0, this.keyLength)), currentKey.getRowLength(),
|
||||
currentKey.getFamilyOffset(), currentKey.getFamilyLength(),
|
||||
currentKey.getQualifierOffset(), currentKey.getQualifierLength(),
|
||||
currentKey.getTimestamp(), currentKey.getTypeByte(), valAndTagsBuffer, vOffset,
|
||||
this.valueLength, memstoreTS, tagsBuf, tOffset, this.tagsLength);
|
||||
|
@ -477,7 +477,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
}
|
||||
}
|
||||
|
||||
protected static class OffheapDecodedCell extends ByteBufferCell implements ExtendedCell {
|
||||
protected static class OffheapDecodedExtendedCell extends ByteBufferExtendedCell {
|
||||
private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
|
||||
+ (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT)
|
||||
+ (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.BYTE_BUFFER));
|
||||
|
@ -497,7 +497,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
private int tagsLength;
|
||||
private long seqId;
|
||||
|
||||
protected OffheapDecodedCell(ByteBuffer keyBuffer, short rowLength, int familyOffset,
|
||||
protected OffheapDecodedExtendedCell(ByteBuffer keyBuffer, short rowLength, int familyOffset,
|
||||
byte familyLength, int qualOffset, int qualLength, long timeStamp, byte typeByte,
|
||||
ByteBuffer valueBuffer, int valueOffset, int valueLen, long seqId, ByteBuffer tagsBuffer,
|
||||
int tagsOffset, int tagsLength) {
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.io.encoding;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -154,10 +154,10 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
}
|
||||
|
||||
private int compareRows(ByteBuffer row, Cell seekCell) {
|
||||
if (seekCell instanceof ByteBufferCell) {
|
||||
if (seekCell instanceof ByteBufferExtendedCell) {
|
||||
return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
|
||||
((ByteBufferCell) seekCell).getRowByteBuffer(),
|
||||
((ByteBufferCell) seekCell).getRowPosition(),
|
||||
((ByteBufferExtendedCell) seekCell).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) seekCell).getRowPosition(),
|
||||
seekCell.getRowLength());
|
||||
} else {
|
||||
return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
|
||||
|
|
|
@ -56,7 +56,7 @@ public class TestByteBufferKeyValue {
|
|||
KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0L, Type.Put, row1);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0L);
|
||||
ByteBufferExtendedCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0L);
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
|
||||
|
@ -179,7 +179,7 @@ public class TestByteBufferKeyValue {
|
|||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getKeyLength());
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), kvCell.getKeyOffset(),
|
||||
kvCell.getKeyLength());
|
||||
ByteBufferCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
|
||||
ByteBufferExtendedCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKeyOnlyKV.getRowByteBuffer(),
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -80,10 +80,10 @@ public class TestTagCompressionContext {
|
|||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
|
||||
TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
|
||||
ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(2);
|
||||
ByteBufferExtendedCell kv1 = (ByteBufferExtendedCell)createOffheapKVWithTags(2);
|
||||
int tagsLength1 = kv1.getTagsLength();
|
||||
context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
|
||||
ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
|
||||
ByteBufferExtendedCell kv2 = (ByteBufferExtendedCell)createOffheapKVWithTags(3);
|
||||
int tagsLength2 = kv2.getTagsLength();
|
||||
context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
|
||||
|
||||
|
@ -129,10 +129,10 @@ public class TestTagCompressionContext {
|
|||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
|
||||
TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
|
||||
ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(1);
|
||||
ByteBufferExtendedCell kv1 = (ByteBufferExtendedCell)createOffheapKVWithTags(1);
|
||||
int tagsLength1 = kv1.getTagsLength();
|
||||
context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
|
||||
ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
|
||||
ByteBufferExtendedCell kv2 = (ByteBufferExtendedCell)createOffheapKVWithTags(3);
|
||||
int tagsLength2 = kv2.getTagsLength();
|
||||
context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.mapreduce.Reducer;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
|
@ -53,7 +53,7 @@ public class CellSortReducer
|
|||
context.setStatus("Read " + map.getClass());
|
||||
int index = 0;
|
||||
for (Cell kv: map) {
|
||||
context.write(row, new MapReduceCell(kv));
|
||||
context.write(row, new MapReduceExtendedCell(kv));
|
||||
if (++index % 100 == 0) context.setStatus("Wrote " + index);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -75,7 +75,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
@ -586,7 +586,7 @@ public class HFileOutputFormat2
|
|||
Class<? extends OutputFormat<?, ?>> cls) throws IOException {
|
||||
Configuration conf = job.getConfiguration();
|
||||
job.setOutputKeyClass(ImmutableBytesWritable.class);
|
||||
job.setOutputValueClass(MapReduceCell.class);
|
||||
job.setOutputValueClass(MapReduceExtendedCell.class);
|
||||
job.setOutputFormatClass(cls);
|
||||
|
||||
if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {
|
||||
|
@ -601,7 +601,7 @@ public class HFileOutputFormat2
|
|||
// sort the incoming values.
|
||||
// TODO it would be nice to pick one or the other of these formats.
|
||||
if (KeyValue.class.equals(job.getMapOutputValueClass())
|
||||
|| MapReduceCell.class.equals(job.getMapOutputValueClass())) {
|
||||
|| MapReduceExtendedCell.class.equals(job.getMapOutputValueClass())) {
|
||||
job.setReducerClass(CellSortReducer.class);
|
||||
} else if (Put.class.equals(job.getMapOutputValueClass())) {
|
||||
job.setReducerClass(PutSortReducer.class);
|
||||
|
@ -661,7 +661,7 @@ public class HFileOutputFormat2
|
|||
Configuration conf = job.getConfiguration();
|
||||
|
||||
job.setOutputKeyClass(ImmutableBytesWritable.class);
|
||||
job.setOutputValueClass(MapReduceCell.class);
|
||||
job.setOutputValueClass(MapReduceExtendedCell.class);
|
||||
job.setOutputFormatClass(HFileOutputFormat2.class);
|
||||
|
||||
ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
@ -61,7 +62,6 @@ import org.apache.hadoop.hbase.client.Table;
|
|||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.io.RawComparator;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
@ -180,7 +180,7 @@ public class Import extends Configured implements Tool {
|
|||
int index = 0;
|
||||
for (Cell kv : kvs) {
|
||||
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),
|
||||
new MapReduceCell(kv));
|
||||
new MapReduceExtendedCell(kv));
|
||||
if (++index % 100 == 0)
|
||||
context.setStatus("Wrote " + index + " KeyValues, "
|
||||
+ "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
|
||||
|
@ -280,7 +280,7 @@ public class Import extends Configured implements Tool {
|
|||
kv = filterKv(filter, kv);
|
||||
// skip if we filtered it out
|
||||
if (kv == null) continue;
|
||||
context.write(row, new MapReduceCell(convertKv(kv, cfRenameMap)));
|
||||
context.write(row, new MapReduceExtendedCell(convertKv(kv, cfRenameMap)));
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -631,7 +631,7 @@ public class Import extends Configured implements Tool {
|
|||
Path outputDir = new Path(hfileOutPath);
|
||||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
job.setMapOutputKeyClass(CellWritableComparable.class);
|
||||
job.setMapOutputValueClass(MapReduceCell.class);
|
||||
job.setMapOutputValueClass(MapReduceExtendedCell.class);
|
||||
job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
|
||||
CellWritableComparable.CellWritableComparator.class,
|
||||
RawComparator.class);
|
||||
|
@ -654,7 +654,7 @@ public class Import extends Configured implements Tool {
|
|||
Path outputDir = new Path(hfileOutPath);
|
||||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
|
||||
job.setMapOutputValueClass(MapReduceCell.class);
|
||||
job.setMapOutputValueClass(MapReduceExtendedCell.class);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
||||
org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.client.Table;
|
|||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
|
@ -111,7 +111,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
continue;
|
||||
}
|
||||
context.write(new ImmutableBytesWritable(CellUtil.cloneRow(cell)),
|
||||
new MapReduceCell(cell));
|
||||
new MapReduceExtendedCell(cell));
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -302,7 +302,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
job.setReducerClass(CellSortReducer.class);
|
||||
Path outputDir = new Path(hfileOutPath);
|
||||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
job.setMapOutputValueClass(MapReduceCell.class);
|
||||
job.setMapOutputValueClass(MapReduceExtendedCell.class);
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table table = conn.getTable(tableName);
|
||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||
|
|
|
@ -20,8 +20,7 @@ package org.apache.hadoop.hbase.util;
|
|||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ExtendedCell;
|
||||
|
@ -32,11 +31,11 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
* A wrapper for a cell to be used with mapreduce, as the output value class for mappers/reducers.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
||||
public class MapReduceExtendedCell extends ByteBufferExtendedCell {
|
||||
|
||||
private final Cell cell;
|
||||
|
||||
public MapReduceCell(Cell cell) {
|
||||
public MapReduceExtendedCell(Cell cell) {
|
||||
this.cell = cell;
|
||||
}
|
||||
|
||||
|
@ -132,8 +131,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public ByteBuffer getRowByteBuffer() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getRowByteBuffer();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getRowByteBuffer();
|
||||
} else {
|
||||
return ByteBuffer.wrap(CellUtil.cloneRow(this.cell));
|
||||
}
|
||||
|
@ -141,8 +140,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public int getRowPosition() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getRowPosition();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getRowPosition();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
|
@ -150,8 +149,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public ByteBuffer getFamilyByteBuffer() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getFamilyByteBuffer();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getFamilyByteBuffer();
|
||||
} else {
|
||||
return ByteBuffer.wrap(CellUtil.cloneFamily(this.cell));
|
||||
}
|
||||
|
@ -159,8 +158,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public int getFamilyPosition() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getFamilyPosition();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getFamilyPosition();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
|
@ -168,8 +167,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public ByteBuffer getQualifierByteBuffer() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getQualifierByteBuffer();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getQualifierByteBuffer();
|
||||
} else {
|
||||
return ByteBuffer.wrap(CellUtil.cloneQualifier(this.cell));
|
||||
}
|
||||
|
@ -177,8 +176,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public int getQualifierPosition() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getQualifierPosition();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getQualifierPosition();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
|
@ -186,8 +185,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public ByteBuffer getValueByteBuffer() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getValueByteBuffer();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getValueByteBuffer();
|
||||
} else {
|
||||
return ByteBuffer.wrap(CellUtil.cloneValue(this.cell));
|
||||
}
|
||||
|
@ -195,8 +194,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public int getValuePosition() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getValuePosition();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getValuePosition();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
|
@ -204,8 +203,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public ByteBuffer getTagsByteBuffer() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getTagsByteBuffer();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getTagsByteBuffer();
|
||||
} else {
|
||||
return ByteBuffer.wrap(CellUtil.cloneTags(this.cell));
|
||||
}
|
||||
|
@ -213,8 +212,8 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
|
||||
@Override
|
||||
public int getTagsPosition() {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) this.cell).getTagsPosition();
|
||||
if (cell instanceof ByteBufferExtendedCell) {
|
||||
return ((ByteBufferExtendedCell) this.cell).getTagsPosition();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
|
@ -39,7 +39,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -674,7 +674,7 @@ public class TestImportExport {
|
|||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArgument(0);
|
||||
MapReduceCell key = (MapReduceCell) invocation.getArgument(1);
|
||||
MapReduceExtendedCell key = (MapReduceExtendedCell) invocation.getArgument(1);
|
||||
assertEquals("Key", Bytes.toString(writer.get()));
|
||||
assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
|
||||
return null;
|
||||
|
|
|
@ -47,12 +47,12 @@ import org.apache.hadoop.hbase.client.Table;
|
|||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.LauncherSecurityManager;
|
||||
import org.apache.hadoop.hbase.util.MapReduceCell;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
|
@ -185,7 +185,7 @@ public class TestWALPlayer {
|
|||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArgument(0);
|
||||
MapReduceCell key = (MapReduceCell) invocation.getArgument(1);
|
||||
MapReduceExtendedCell key = (MapReduceExtendedCell) invocation.getArgument(1);
|
||||
assertEquals("row", Bytes.toString(writer.get()));
|
||||
assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
|
||||
return null;
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -383,14 +383,15 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
+ CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
|
||||
}
|
||||
byte[] midRow;
|
||||
boolean bufferBacked = left instanceof ByteBufferCell && right instanceof ByteBufferCell;
|
||||
boolean bufferBacked = left instanceof ByteBufferExtendedCell
|
||||
&& right instanceof ByteBufferExtendedCell;
|
||||
if (diff < 0) {
|
||||
// Left row is < right row.
|
||||
if (bufferBacked) {
|
||||
midRow = getMinimumMidpointArray(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
|
||||
((ByteBufferCell) right).getRowByteBuffer(),
|
||||
((ByteBufferCell) right).getRowPosition(), right.getRowLength());
|
||||
midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
|
||||
((ByteBufferExtendedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength());
|
||||
} else {
|
||||
midRow = getMinimumMidpointArray(left.getRowArray(), left.getRowOffset(),
|
||||
left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
|
@ -407,10 +408,10 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
}
|
||||
if (diff < 0) {
|
||||
if (bufferBacked) {
|
||||
midRow = getMinimumMidpointArray(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
|
||||
midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
|
||||
} else {
|
||||
midRow = getMinimumMidpointArray(left.getFamilyArray(), left.getFamilyOffset(),
|
||||
left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
|
||||
|
@ -429,10 +430,10 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
}
|
||||
if (diff < 0) {
|
||||
if (bufferBacked) {
|
||||
midRow = getMinimumMidpointArray(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(), right.getQualifierLength());
|
||||
midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
|
||||
} else {
|
||||
midRow = getMinimumMidpointArray(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
|
||||
|
|
|
@ -31,12 +31,12 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
|||
// to be serialized
|
||||
// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
|
||||
@InterfaceAudience.Private
|
||||
public class ByteBufferChunkCell extends ByteBufferKeyValue {
|
||||
public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
|
||||
public class ByteBufferChunkKeyValue extends ByteBufferKeyValue {
|
||||
public ByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
super(buf, offset, length);
|
||||
}
|
||||
|
||||
public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
public ByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
super(buf, offset, length, seqId);
|
||||
}
|
||||
|
|
@ -122,6 +122,6 @@ public class CellChunkMap extends CellFlatMap {
|
|||
+ chunk.isFromPool() + ". We were looking for a cell at index " + i);
|
||||
}
|
||||
|
||||
return new ByteBufferChunkCell(buf, offsetOfCell, lengthOfCell, cellSeqID);
|
||||
return new ByteBufferChunkKeyValue(buf, offsetOfCell, lengthOfCell, cellSeqID);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -159,9 +159,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
|
|||
// which directly return tagsLen as 0. So we avoid parsing many length components in
|
||||
// reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
|
||||
// call getTagsLength().
|
||||
return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
|
||||
return new NoTagByteBufferChunkKeyValue(buf, offset, len, cell.getSequenceId());
|
||||
} else {
|
||||
return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
|
||||
return new ByteBufferChunkKeyValue(buf, offset, len, cell.getSequenceId());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -29,13 +29,13 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
|||
* @see MemStoreLAB
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
|
||||
public class NoTagByteBufferChunkKeyValue extends NoTagsByteBufferKeyValue {
|
||||
|
||||
public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
|
||||
public NoTagByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
super(buf, offset, length);
|
||||
}
|
||||
|
||||
public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
public NoTagByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
super(buf, offset, length, seqId);
|
||||
}
|
||||
|
|
@ -46,7 +46,7 @@ import java.util.concurrent.atomic.LongAdder;
|
|||
import org.apache.commons.lang3.mutable.MutableObject;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -1295,8 +1295,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
// 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.
|
||||
// So we make a guess.
|
||||
if (c instanceof ByteBufferCell) {
|
||||
ByteBufferCell bbCell = (ByteBufferCell) c;
|
||||
if (c instanceof ByteBufferExtendedCell) {
|
||||
ByteBufferExtendedCell bbCell = (ByteBufferExtendedCell) c;
|
||||
ByteBuffer bb = bbCell.getValueByteBuffer();
|
||||
if (bb != lastBlock) {
|
||||
context.incrementResponseBlockSize(bb.capacity());
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
|
||||
import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OffheapDecodedCell;
|
||||
import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell;
|
||||
import org.apache.hadoop.hbase.codec.Codec.Decoder;
|
||||
import org.apache.hadoop.hbase.codec.Codec.Encoder;
|
||||
|
@ -108,7 +107,8 @@ public class TestBufferedDataBlockEncoder {
|
|||
kv2.getTagsLength());
|
||||
KeyValue kv3 = new KeyValue(Bytes.toBytes("r3"), Bytes.toBytes("cf"), Bytes.toBytes("qual"),
|
||||
HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"));
|
||||
OffheapDecodedCell c3 = new OffheapDecodedCell(ByteBuffer.wrap(kv2.getKey()),
|
||||
BufferedDataBlockEncoder.OffheapDecodedExtendedCell
|
||||
c3 = new BufferedDataBlockEncoder.OffheapDecodedExtendedCell(ByteBuffer.wrap(kv2.getKey()),
|
||||
kv2.getRowLength(), kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(),
|
||||
kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(),
|
||||
kv2.getTimestamp(), kv2.getTypeByte(), ByteBuffer.wrap(kv2.getValueArray()),
|
||||
|
|
Loading…
Reference in New Issue