HBASE-19629 RawCell#getTags should return the Iterator<Tag> in order to avoid iterating through whole tag array at once
Signed-off-by: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
7145d98182
commit
467a4667d8
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
|
@ -33,7 +31,6 @@ import java.util.Optional;
|
|||
import java.util.TreeMap;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
|
@ -60,7 +57,6 @@ 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.common.collect.ArrayListMultimap;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataInput;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataOutput;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteStreams;
|
||||
|
@ -915,35 +911,23 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
|||
if (cell instanceof RawCell) {
|
||||
return ((RawCell) cell).getTag(type);
|
||||
}
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional.of(new ArrayBackedTag(getTagsArray(), pos,
|
||||
tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.empty();
|
||||
return PrivateCellUtil.getTag(cell, type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
public Iterator<Tag> getTags() {
|
||||
if (cell instanceof RawCell) {
|
||||
return ((RawCell) cell).getTags();
|
||||
}
|
||||
return Lists.newArrayList(PrivateCellUtil.tagsIterator(cell));
|
||||
return PrivateCellUtil.tagsIterator(cell);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] cloneTags() {
|
||||
if (cell instanceof RawCell) {
|
||||
return ((RawCell) cell).cloneTags();
|
||||
} else {
|
||||
return PrivateCellUtil.cloneTags(cell);
|
||||
}
|
||||
return PrivateCellUtil.cloneTags(cell);
|
||||
}
|
||||
|
||||
private long heapOverhead() {
|
||||
|
|
|
@ -17,16 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
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;
|
||||
|
@ -350,31 +343,4 @@ public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
|
|||
hash = 31 * hash + cell.getTypeByte();
|
||||
return hash;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsPosition();
|
||||
int pos = offset;
|
||||
int tagLen;
|
||||
while (pos < offset + length) {
|
||||
ByteBuffer tagsBuffer = getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return Optional.ofNullable(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,13 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.commons.lang3.ArrayUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
|
@ -304,30 +297,4 @@ public class IndividualBytesFieldCell implements ExtendedCell {
|
|||
public String toString() {
|
||||
return CellUtil.toString(this, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
|
||||
import static org.apache.hadoop.hbase.util.Bytes.len;
|
||||
|
||||
import java.io.DataInput;
|
||||
|
@ -33,8 +32,6 @@ import java.util.HashMap;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
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;
|
||||
|
@ -1165,11 +1162,11 @@ public class KeyValue implements ExtendedCell {
|
|||
Bytes.toStringBinary(getQualifierArray(), getQualifierOffset(), getQualifierLength()));
|
||||
stringMap.put("timestamp", getTimestamp());
|
||||
stringMap.put("vlen", getValueLength());
|
||||
List<Tag> tags = getTags();
|
||||
Iterator<Tag> tags = getTags();
|
||||
if (tags != null) {
|
||||
List<String> tagsString = new ArrayList<>(tags.size());
|
||||
for (Tag t : tags) {
|
||||
tagsString.add(t.toString());
|
||||
List<String> tagsString = new ArrayList<String>();
|
||||
while (tags.hasNext()) {
|
||||
tagsString.add(tags.next().toString());
|
||||
}
|
||||
stringMap.put("tag", tagsString);
|
||||
}
|
||||
|
@ -2555,30 +2552,4 @@ public class KeyValue implements ExtendedCell {
|
|||
kv.setSequenceId(this.getSequenceId());
|
||||
return kv;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.hadoop.hbase;
|
|||
import static org.apache.hadoop.hbase.HConstants.EMPTY_BYTE_ARRAY;
|
||||
import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.DataOutput;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
|
@ -32,7 +30,6 @@ import java.util.ArrayList;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
|
@ -45,6 +42,7 @@ 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.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Utility methods helpful slinging {@link Cell} instances. It has more powerful and
|
||||
|
@ -314,32 +312,6 @@ public final class PrivateCellUtil {
|
|||
Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
|
||||
return new TagRewriteCell(clonedBaseCell, this.tags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
||||
static class TagRewriteByteBufferCell extends ByteBufferCell implements ExtendedCell {
|
||||
|
@ -573,33 +545,6 @@ public final class PrivateCellUtil {
|
|||
public int getTagsPosition() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsPosition();
|
||||
int pos = offset;
|
||||
int tagLen;
|
||||
while (pos < offset + length) {
|
||||
ByteBuffer tagsBuffer = getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return Optional.ofNullable(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
||||
static class ValueAndTagRewriteCell extends TagRewriteCell {
|
||||
|
@ -952,18 +897,18 @@ public final class PrivateCellUtil {
|
|||
ByteBuffer tagsBuffer = ((ByteBufferCell) cell).getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return Optional.ofNullable(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
return Optional.of(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
} else {
|
||||
tagLen = Bytes.readAsInt(cell.getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (cell.getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
.of(new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1424,32 +1369,6 @@ public final class PrivateCellUtil {
|
|||
public int getTagsLength() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1603,33 +1522,6 @@ public final class PrivateCellUtil {
|
|||
public int getValuePosition() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsPosition();
|
||||
int pos = offset;
|
||||
int tagLen;
|
||||
while (pos < offset + length) {
|
||||
ByteBuffer tagsBuffer = getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return Optional.ofNullable(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
||||
private static class FirstOnRowCell extends EmptyCell {
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -41,14 +41,18 @@ public interface RawCell extends Cell {
|
|||
* Creates a list of tags in the current cell
|
||||
* @return a list of tags
|
||||
*/
|
||||
List<Tag> getTags();
|
||||
default Iterator<Tag> getTags() {
|
||||
return PrivateCellUtil.tagsIterator(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the specific tag of the given type
|
||||
* @param type the type of the tag
|
||||
* @return the specific tag if available or null
|
||||
*/
|
||||
Optional<Tag> getTag(byte type);
|
||||
default Optional<Tag> getTag(byte type) {
|
||||
return PrivateCellUtil.getTag(this, type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check the length of tags. If it is invalid, throw IllegalArgumentException
|
||||
|
|
|
@ -21,14 +21,8 @@ import java.io.DataOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferTag;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -38,7 +32,6 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.io.TagCompressionContext;
|
||||
import org.apache.hadoop.hbase.io.util.LRUDictionary;
|
||||
import org.apache.hadoop.hbase.io.util.StreamUtils;
|
||||
|
@ -482,32 +475,6 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
// This is not used in actual flow. Throwing UnsupportedOperationException
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, Tag.TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + Tag.TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(getTagsArray(), pos, tagLen + Tag.TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += Tag.TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
||||
protected static class OffheapDecodedCell extends ByteBufferCell implements ExtendedCell {
|
||||
|
@ -753,35 +720,6 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
// This is not used in actual flow. Throwing UnsupportedOperationException
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsPosition();
|
||||
int pos = offset;
|
||||
int tagLen;
|
||||
while (pos < offset + length) {
|
||||
ByteBuffer tagsBuffer = getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, Tag.TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + Tag.TAG_LENGTH_SIZE) == type) {
|
||||
return Optional
|
||||
.ofNullable(new ByteBufferTag(tagsBuffer, pos, tagLen + Tag.TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += Tag.TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
protected abstract static class BufferedEncodedSeeker<STATE extends SeekerState>
|
||||
|
|
|
@ -158,7 +158,7 @@ public class TestByteBufferKeyValue {
|
|||
assertEquals(0L, offheapKV.getTimestamp());
|
||||
assertEquals(Type.Put.getCode(), offheapKV.getTypeByte());
|
||||
// change tags to handle both onheap and offheap stuff
|
||||
List<Tag> resTags = offheapKV.getTags();
|
||||
List<Tag> resTags = PrivateCellUtil.getTags(offheapKV);
|
||||
Tag tag1 = resTags.get(0);
|
||||
assertEquals(t1.getType(), tag1.getType());
|
||||
assertEquals(Tag.getValueAsString(t1),
|
||||
|
|
|
@ -453,7 +453,7 @@ public class TestKeyValue extends TestCase {
|
|||
kv.getQualifierLength(), q, 0, q.length));
|
||||
assertTrue(Bytes.equals(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(), value, 0,
|
||||
value.length));
|
||||
List<Tag> tags = kv.getTags();
|
||||
List<Tag> tags = PrivateCellUtil.getTags(kv);
|
||||
assertNotNull(tags);
|
||||
assertEquals(2, tags.size());
|
||||
boolean meta1Ok = false, meta2Ok = false;
|
||||
|
|
|
@ -27,13 +27,12 @@ import java.io.DataInputStream;
|
|||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -80,7 +79,7 @@ public class TestCellCodecWithTags {
|
|||
Cell c = decoder.current();
|
||||
assertTrue(CellUtil.equals(c, cell1));
|
||||
List<Tag> tags =
|
||||
((RawCell)c).getTags();
|
||||
PrivateCellUtil.getTags(c);
|
||||
assertEquals(2, tags.size());
|
||||
Tag tag = tags.get(0);
|
||||
assertEquals(1, tag.getType());
|
||||
|
@ -91,7 +90,7 @@ public class TestCellCodecWithTags {
|
|||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellUtil.equals(c, cell2));
|
||||
tags = ((RawCell)c).getTags();
|
||||
tags = PrivateCellUtil.getTags(c);
|
||||
assertEquals(1, tags.size());
|
||||
tag = tags.get(0);
|
||||
assertEquals(1, tag.getType());
|
||||
|
@ -99,7 +98,7 @@ public class TestCellCodecWithTags {
|
|||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellUtil.equals(c, cell3));
|
||||
tags = ((RawCell)c).getTags();
|
||||
tags = PrivateCellUtil.getTags(c);
|
||||
assertEquals(3, tags.size());
|
||||
tag = tags.get(0);
|
||||
assertEquals(2, tag.getType());
|
||||
|
|
|
@ -27,13 +27,12 @@ import java.io.DataInputStream;
|
|||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -80,7 +79,7 @@ public class TestKeyValueCodecWithTags {
|
|||
Cell c = decoder.current();
|
||||
assertTrue(CellUtil.equals(c, kv1));
|
||||
List<Tag> tags =
|
||||
((RawCell)c).getTags();
|
||||
PrivateCellUtil.getTags(c);
|
||||
assertEquals(2, tags.size());
|
||||
Tag tag = tags.get(0);
|
||||
assertEquals(1, tag.getType());
|
||||
|
@ -91,7 +90,7 @@ public class TestKeyValueCodecWithTags {
|
|||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellUtil.equals(c, kv2));
|
||||
tags = ((RawCell)c).getTags();
|
||||
tags = PrivateCellUtil.getTags(c);
|
||||
assertEquals(1, tags.size());
|
||||
tag = tags.get(0);
|
||||
assertEquals(1, tag.getType());
|
||||
|
@ -99,7 +98,7 @@ public class TestKeyValueCodecWithTags {
|
|||
assertTrue(decoder.advance());
|
||||
c = decoder.current();
|
||||
assertTrue(CellUtil.equals(c, kv3));
|
||||
tags = ((RawCell)c).getTags();
|
||||
tags = PrivateCellUtil.getTags(c);
|
||||
assertEquals(3, tags.size());
|
||||
tag = tags.get(0);
|
||||
assertEquals(2, tag.getType());
|
||||
|
|
|
@ -17,23 +17,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ExtendedCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -276,30 +268,4 @@ public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Tag> getTag(byte type) {
|
||||
int length = getTagsLength();
|
||||
int offset = getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = Bytes.readAsInt(getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
if (getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
|
||||
return Optional
|
||||
.ofNullable(new ArrayBackedTag(getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
|
||||
}
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return Optional.ofNullable(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Tag> getTags() {
|
||||
List<Tag> tags = new ArrayList<>();
|
||||
Iterator<Tag> tagsItr = PrivateCellUtil.tagsIterator(this);
|
||||
while (tagsItr.hasNext()) {
|
||||
tags.add(tagsItr.next());
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -61,6 +60,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.HadoopShims;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.PerformanceEvaluation;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
|
@ -491,7 +491,7 @@ public class TestHFileOutputFormat2 {
|
|||
HFileScanner scanner = reader.getScanner(false, false, false);
|
||||
scanner.seekTo();
|
||||
Cell cell = scanner.getCell();
|
||||
List<Tag> tagsFromCell = ((RawCell)cell).getTags();
|
||||
List<Tag> tagsFromCell = PrivateCellUtil.getTags(cell);
|
||||
assertTrue(tagsFromCell.size() > 0);
|
||||
for (Tag tag : tagsFromCell) {
|
||||
assertTrue(tag.getType() == TagType.TTL_TAG_TYPE);
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -399,7 +398,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
|
|||
+ Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
|
||||
cell.getValueLength()));
|
||||
int i = 0;
|
||||
List<Tag> tags = ((RawCell)cell).getTags();
|
||||
List<Tag> tags = PrivateCellUtil.getTags(cell);
|
||||
for (Tag tag : tags) {
|
||||
out.print(String.format(" T[%d]: %s", i++, tag.toString()));
|
||||
}
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -87,7 +88,7 @@ public class TestStoreFileScannerWithTagCompression {
|
|||
byte[] key5 = Bytes.toBytes("k5");
|
||||
assertTrue(Bytes.equals(key5, 0, key5.length, kv.getRowArray(), kv.getRowOffset(),
|
||||
kv.getRowLength()));
|
||||
List<Tag> tags = KeyValueUtil.ensureKeyValue(kv).getTags();
|
||||
List<Tag> tags = PrivateCellUtil.getTags(kv);
|
||||
assertEquals(1, tags.size());
|
||||
assertEquals("tag3", Bytes.toString(Tag.cloneValue(tags.get(0))));
|
||||
} finally {
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
@ -613,7 +613,7 @@ public class TestTags {
|
|||
CellScanner cellScanner = result.cellScanner();
|
||||
if (cellScanner.advance()) {
|
||||
Cell cell = cellScanner.current();
|
||||
tags = ((RawCell)cell).getTags();
|
||||
tags = PrivateCellUtil.getTags(cell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
|
|||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.codec.Codec.Decoder;
|
||||
import org.apache.hadoop.hbase.codec.Codec.Encoder;
|
||||
|
@ -81,16 +82,16 @@ public class TestWALCellCodecWithCompression {
|
|||
Decoder decoder = codec.getDecoder(is);
|
||||
decoder.advance();
|
||||
KeyValue kv = (KeyValue) decoder.current();
|
||||
List<Tag> tags = kv.getTags();
|
||||
List<Tag> tags = PrivateCellUtil.getTags(kv);
|
||||
assertEquals(1, tags.size());
|
||||
assertEquals("tagValue1", Bytes.toString(Tag.cloneValue(tags.get(0))));
|
||||
decoder.advance();
|
||||
kv = (KeyValue) decoder.current();
|
||||
tags = kv.getTags();
|
||||
tags = PrivateCellUtil.getTags(kv);
|
||||
assertEquals(0, tags.size());
|
||||
decoder.advance();
|
||||
kv = (KeyValue) decoder.current();
|
||||
tags = kv.getTags();
|
||||
tags = PrivateCellUtil.getTags(kv);
|
||||
assertEquals(2, tags.size());
|
||||
assertEquals("tagValue1", Bytes.toString(Tag.cloneValue(tags.get(0))));
|
||||
assertEquals("tagValue2", Bytes.toString(Tag.cloneValue(tags.get(1))));
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
@ -252,7 +252,7 @@ public class TestReplicationWithTags {
|
|||
// Check tag presence in the 1st cell in 1st Result
|
||||
if (!results.isEmpty()) {
|
||||
Cell cell = results.get(0);
|
||||
tags = ((RawCell)cell).getTags();
|
||||
tags = PrivateCellUtil.getTags(cell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.RawCell;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
|
@ -285,7 +284,7 @@ public class TestVisibilityLabelsReplication {
|
|||
for (Cell cell : cells) {
|
||||
if ((Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), row, 0,
|
||||
row.length))) {
|
||||
List<Tag> tags = ((RawCell)cell).getTags();
|
||||
List<Tag> tags = PrivateCellUtil.getTags(cell);
|
||||
for (Tag tag : tags) {
|
||||
if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
|
||||
assertEquals(visTag, Tag.getValueAsString(tag));
|
||||
|
@ -413,9 +412,9 @@ public class TestVisibilityLabelsReplication {
|
|||
cf = CellUtil.cloneFamily(kv);
|
||||
}
|
||||
Tag tag = new ArrayBackedTag((byte) NON_VIS_TAG_TYPE, attribute);
|
||||
List<Tag> tagList = new ArrayList<>(kv.getTags().size() + 1);
|
||||
List<Tag> tagList = new ArrayList<>(PrivateCellUtil.getTags(cell).size() + 1);
|
||||
tagList.add(tag);
|
||||
tagList.addAll(kv.getTags());
|
||||
tagList.addAll(PrivateCellUtil.getTags(cell));
|
||||
Cell newcell = PrivateCellUtil.createCell(kv, tagList);
|
||||
((List<Cell>) updatedCells).add(newcell);
|
||||
}
|
||||
|
@ -442,7 +441,7 @@ public class TestVisibilityLabelsReplication {
|
|||
// Check tag presence in the 1st cell in 1st Result
|
||||
if (!results.isEmpty()) {
|
||||
Cell cell = results.get(0);
|
||||
tags = ((RawCell)cell).getTags();
|
||||
tags = PrivateCellUtil.getTags(cell);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue