HBASE-19628 ByteBufferCell should extend ExtendedCell
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6b39062e Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6b39062e Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6b39062e Branch: refs/heads/master Commit: 6b39062e8667e674fb9d5a7a3578fe7d21e7ef05 Parents: 467a466 Author: Chia-Ping Tsai <chia7...@gmail.com> Authored: Wed Dec 27 22:09:42 2017 +0800 Committer: Chia-Ping Tsai <chia7...@gmail.com> Committed: Wed Dec 27 22:26:40 2017 +0800 ---------------------------------------------------------------------- .../mapreduce/MapReduceHFileSplitterJob.java | 7 +- .../hadoop/hbase/filter/ColumnPrefixFilter.java | 8 +- .../hadoop/hbase/filter/KeyOnlyFilter.java | 57 ++- .../hadoop/hbase/filter/PrefixFilter.java | 8 +- .../hbase/shaded/protobuf/ProtobufUtil.java | 20 +- .../hadoop/hbase/filter/TestKeyOnlyFilter.java | 6 +- .../org/apache/hadoop/hbase/ByteBufferCell.java | 120 ----- .../hadoop/hbase/ByteBufferExtendedCell.java | 121 +++++ .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java | 44 +- .../apache/hadoop/hbase/ByteBufferKeyValue.java | 6 +- .../apache/hadoop/hbase/CellComparatorImpl.java | 72 +-- .../java/org/apache/hadoop/hbase/CellUtil.java | 181 +++---- .../hadoop/hbase/ExtendedCellBuilder.java | 2 +- .../apache/hadoop/hbase/PrivateCellUtil.java | 502 ++++++++++--------- .../java/org/apache/hadoop/hbase/TagUtil.java | 6 +- .../io/encoding/BufferedDataBlockEncoder.java | 14 +- .../hbase/io/encoding/RowIndexSeekerV1.java | 8 +- .../hadoop/hbase/TestByteBufferKeyValue.java | 4 +- .../hbase/io/TestTagCompressionContext.java | 10 +- .../hadoop/hbase/mapreduce/CellSortReducer.java | 4 +- .../hbase/mapreduce/HFileOutputFormat2.java | 8 +- .../apache/hadoop/hbase/mapreduce/Import.java | 10 +- .../hadoop/hbase/mapreduce/WALPlayer.java | 6 +- .../apache/hadoop/hbase/util/MapReduceCell.java | 271 ---------- .../hbase/util/MapReduceExtendedCell.java | 270 ++++++++++ .../hbase/mapreduce/TestImportExport.java | 4 +- .../hadoop/hbase/mapreduce/TestWALPlayer.java | 4 +- .../hadoop/hbase/io/hfile/HFileWriterImpl.java | 29 +- .../hbase/regionserver/ByteBufferChunkCell.java | 48 -- .../regionserver/ByteBufferChunkKeyValue.java | 48 ++ .../hadoop/hbase/regionserver/CellChunkMap.java | 2 +- .../hbase/regionserver/MemStoreLABImpl.java | 4 +- .../regionserver/NoTagByteBufferChunkCell.java | 48 -- .../NoTagByteBufferChunkKeyValue.java | 48 ++ .../hbase/regionserver/RSRpcServices.java | 6 +- .../encoding/TestBufferedDataBlockEncoder.java | 4 +- 36 files changed, 1059 insertions(+), 951 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java ---------------------------------------------------------------------- diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java index 31428d0..cb5df5e 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java @@ -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)) { http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java index 4811691..b4e7a0f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java @@ -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); http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java index 32286b6..4487292 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java @@ -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; } @@ -309,6 +314,21 @@ public class KeyOnlyFilter extends FilterBase { } @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()); + } } } http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java index 161c1a5..09a3304 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -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(), http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 17b1141..8f44135 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -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( http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java index f957b59..28fe2bf 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java @@ -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)); http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java deleted file mode 100644 index b3be0ac..0000000 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java +++ /dev/null @@ -1,120 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase; - -import java.nio.ByteBuffer; - -import org.apache.yetus.audience.InterfaceAudience; - -/** - * This class is a server side extension to the {@link Cell} interface. It is used when the - * Cell is backed by a {@link ByteBuffer}: i.e. <code>cell instanceof ByteBufferedCell</code>. - * - * <p>This class has getters for the row, column family, column qualifier, value and tags hosting - * ByteBuffers. It also has getters of the *position* within a ByteBuffer where these - * field bytes begin. These are needed because a single ByteBuffer may back one or many Cell - * instances -- it depends on the implementation -- so the ByteBuffer position as returned by - * {@link ByteBuffer#arrayOffset()} cannot be relied upon. Also, do not confuse these position - * methods with the getXXXOffset methods from the super Interface, {@link Cell}; dependent up on - * implementation, the Cell getXXXOffset methods can return the same value as a call to its - * equivalent position method from below BUT they can also stray; if a ByteBufferedCell, use the - * below position methods to find where a field begins. - * - * <p>Use the getXXXLength methods from Cell to find a fields length. - * - * <p>A Cell object can be of this type only on the server side. - * - * <p>WARNING: If a Cell is backed by an offheap ByteBuffer, any call to getXXXArray() will result - * in a temporary byte array creation and a bytes copy. Avoid these allocations by using the - * appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell - * 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) { - .... - } - if (left instanceof ByteBufferCell) { - .... - } - if (right instanceof ByteBufferCell) { - .... - } - 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. - */ -@InterfaceAudience.Private -public abstract class ByteBufferCell implements Cell { - /** - * @return The {@link ByteBuffer} containing the row bytes. - */ - public abstract ByteBuffer getRowByteBuffer(); - - /** - * @return Position in the {@link ByteBuffer} where row bytes start - */ - public abstract int getRowPosition(); - - /** - * @return The {@link ByteBuffer} containing the column family bytes. - */ - public abstract ByteBuffer getFamilyByteBuffer(); - - /** - * @return Position in the {@link ByteBuffer} where column family bytes start - */ - public abstract int getFamilyPosition(); - - /** - * @return The {@link ByteBuffer} containing the column qualifier bytes. - */ - public abstract ByteBuffer getQualifierByteBuffer(); - - /** - * @return Position in the {@link ByteBuffer} where column qualifier bytes start - */ - public abstract int getQualifierPosition(); - - /** - * @return The {@link ByteBuffer} containing the value bytes. - */ - public abstract ByteBuffer getValueByteBuffer(); - - /** - * @return Position in the {@link ByteBuffer} where value bytes start - */ - public abstract int getValuePosition(); - - /** - * @return The {@link ByteBuffer} containing the tag bytes. - */ - public abstract ByteBuffer getTagsByteBuffer(); - - /** - * @return Position in the {@link ByteBuffer} where tag bytes start - */ - public abstract int getTagsPosition(); -} http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java new file mode 100644 index 0000000..3e4cc95 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase; + +import java.nio.ByteBuffer; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * This class is a server side extension to the {@link Cell} interface. It is used when the + * Cell is backed by a {@link ByteBuffer}: i.e. <code>cell instanceof ByteBufferedCell</code>. + * + * <p>This class has getters for the row, column family, column qualifier, value and tags hosting + * ByteBuffers. It also has getters of the *position* within a ByteBuffer where these + * field bytes begin. These are needed because a single ByteBuffer may back one or many Cell + * instances -- it depends on the implementation -- so the ByteBuffer position as returned by + * {@link ByteBuffer#arrayOffset()} cannot be relied upon. Also, do not confuse these position + * methods with the getXXXOffset methods from the super Interface, {@link Cell}; dependent up on + * implementation, the Cell getXXXOffset methods can return the same value as a call to its + * equivalent position method from below BUT they can also stray; if a ByteBufferedCell, use the + * below position methods to find where a field begins. + * + * <p>Use the getXXXLength methods from Cell to find a fields length. + * + * <p>A Cell object can be of this type only on the server side. + * + * <p>WARNING: If a Cell is backed by an offheap ByteBuffer, any call to getXXXArray() will result + * in a temporary byte array creation and a bytes copy. Avoid these allocations by using the + * appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell + * when it is not. + */ +/* + * 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 ByteBufferExtendedCell) { + .... + } + 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 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 ByteBufferExtendedCell implements ExtendedCell { + /** + * @return The {@link ByteBuffer} containing the row bytes. + */ + public abstract ByteBuffer getRowByteBuffer(); + + /** + * @return Position in the {@link ByteBuffer} where row bytes start + */ + public abstract int getRowPosition(); + + /** + * @return The {@link ByteBuffer} containing the column family bytes. + */ + public abstract ByteBuffer getFamilyByteBuffer(); + + /** + * @return Position in the {@link ByteBuffer} where column family bytes start + */ + public abstract int getFamilyPosition(); + + /** + * @return The {@link ByteBuffer} containing the column qualifier bytes. + */ + public abstract ByteBuffer getQualifierByteBuffer(); + + /** + * @return Position in the {@link ByteBuffer} where column qualifier bytes start + */ + public abstract int getQualifierPosition(); + + /** + * @return The {@link ByteBuffer} containing the value bytes. + */ + public abstract ByteBuffer getValueByteBuffer(); + + /** + * @return Position in the {@link ByteBuffer} where value bytes start + */ + public abstract int getValuePosition(); + + /** + * @return The {@link ByteBuffer} containing the tag bytes. + */ + public abstract ByteBuffer getTagsByteBuffer(); + + /** + * @return Position in the {@link ByteBuffer} where tag bytes start + */ + public abstract int getTagsPosition(); +} http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java index 713314e..7ecb946 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java @@ -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. @@ -152,6 +157,21 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell { } @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); + } } http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java index 798180a..54af38f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java @@ -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(), http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java index 771fdaa..f1abbef 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -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, http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index f320083..d6dadca 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -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(), http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java index b964d67..0afe273 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java @@ -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 {