HBASE-13387 Add ByteBufferedCell an extension to Cell.
This commit is contained in:
parent
41c8ec7aea
commit
d07ff5ec5a
|
@ -21,7 +21,7 @@ MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
|
|||
OK_RELEASEAUDIT_WARNINGS=0
|
||||
# Allow four warnings. Javadoc complains about sun.misc.Unsafe use.
|
||||
# See HBASE-7457, HBASE-13761
|
||||
OK_JAVADOC_WARNINGS=6
|
||||
OK_JAVADOC_WARNINGS=7
|
||||
|
||||
MAX_LINE_LENGTH=100
|
||||
|
||||
|
|
|
@ -46,8 +46,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
|
|||
if (kv == null || CellUtil.cloneValue(kv) == null) {
|
||||
return null;
|
||||
}
|
||||
return Bytes.toBigDecimal(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()).
|
||||
setScale(2, RoundingMode.HALF_EVEN);
|
||||
return CellUtil.getValueAsBigDecimal(kv).setScale(2, RoundingMode.HALF_EVEN);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client.coprocessor;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
|
@ -46,7 +47,7 @@ public class DoubleColumnInterpreter extends ColumnInterpreter<Double, Double,
|
|||
throws IOException {
|
||||
if (c == null || c.getValueLength() != Bytes.SIZEOF_DOUBLE)
|
||||
return null;
|
||||
return Bytes.toDouble(c.getValueArray(), c.getValueOffset());
|
||||
return CellUtil.getValueAsDouble(c);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client.coprocessor;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
|
@ -43,7 +44,7 @@ public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
|
|||
throws IOException {
|
||||
if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG)
|
||||
return null;
|
||||
return Bytes.toLong(kv.getValueArray(), kv.getValueOffset());
|
||||
return CellUtil.getValueAsLong(kv);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,8 +22,7 @@ import java.io.DataInput;
|
|||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class TablePermission extends Permission {
|
||||
private static final Log LOG = LogFactory.getLog(TablePermission.class);
|
||||
|
||||
private TableName table;
|
||||
private byte[] family;
|
||||
|
@ -248,15 +246,11 @@ public class TablePermission extends Permission {
|
|||
return false;
|
||||
}
|
||||
|
||||
if (family != null &&
|
||||
(Bytes.compareTo(family, 0, family.length,
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength()) != 0)) {
|
||||
if (family != null && !(CellUtil.matchingFamily(kv, family))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (qualifier != null &&
|
||||
(Bytes.compareTo(qualifier, 0, qualifier.length,
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()) != 0)) {
|
||||
if (qualifier != null && !(CellUtil.matchingQualifier(kv, qualifier))) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,85 @@
|
|||
/**
|
||||
* 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.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* This class is a server side extension to the Cell interface. This is used when the actual Cell
|
||||
* implementation is backed by {@link ByteBuffer}. This class contain ByteBuffer backed getters for
|
||||
* row, cf, qualifier, value and tags. Also getters of the position where these field bytes begin. A
|
||||
* cell object can be of this type only in server side. When the object is of this type, use the
|
||||
* getXXXByteBuffer() method along with getXXXPositionInByteBuffer(). If cell is backed by off heap
|
||||
* ByteBuffer the call to getXXXArray() will result is temporary byte array creation and bytes copy
|
||||
* resulting in lot of garbage.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ByteBufferedCell implements Cell {
|
||||
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the row bytes.
|
||||
*/
|
||||
abstract ByteBuffer getRowByteBuffer();
|
||||
|
||||
/**
|
||||
* @return Position in the {@link ByteBuffer} where row bytes start
|
||||
*/
|
||||
abstract int getRowPositionInByteBuffer();
|
||||
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the column family bytes.
|
||||
*/
|
||||
abstract ByteBuffer getFamilyByteBuffer();
|
||||
|
||||
/**
|
||||
* @return Position in the {@link ByteBuffer} where column family bytes start
|
||||
*/
|
||||
abstract int getFamilyPositionInByteBuffer();
|
||||
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the column qualifier bytes.
|
||||
*/
|
||||
abstract ByteBuffer getQualifierByteBuffer();
|
||||
|
||||
/**
|
||||
* @return Position in the {@link ByteBuffer} where column qualifier bytes start
|
||||
*/
|
||||
abstract int getQualifierPositionInByteBuffer();
|
||||
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the value bytes.
|
||||
*/
|
||||
abstract ByteBuffer getValueByteBuffer();
|
||||
|
||||
/**
|
||||
* @return Position in the {@link ByteBuffer} where value bytes start
|
||||
*/
|
||||
abstract int getValuePositionInByteBuffer();
|
||||
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the tag bytes.
|
||||
*/
|
||||
abstract ByteBuffer getTagsByteBuffer();
|
||||
|
||||
/**
|
||||
* @return Position in the {@link ByteBuffer} where tag bytes start
|
||||
*/
|
||||
abstract int getTagsPositionInByteBuffer();
|
||||
}
|
|
@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.common.primitives.Longs;
|
||||
|
@ -36,8 +37,8 @@ import com.google.common.primitives.Longs;
|
|||
* takes account of the special formatting of the row where we have commas to delimit table from
|
||||
* regionname, from row. See KeyValue for how it has a special comparator to do hbase:meta cells
|
||||
* and yet another for -ROOT-.
|
||||
* While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells format
|
||||
* should be taken into consideration, for which the instance of this comparator
|
||||
* While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells
|
||||
* format should be taken into consideration, for which the instance of this comparator
|
||||
* should be used. In all other cases the static APIs in this comparator would be enough
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
|
||||
|
@ -90,8 +91,6 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return an int greater than 0 if left is greater than right
|
||||
* lesser than 0 if left is lesser than right
|
||||
* equal to 0 if left is equal to right
|
||||
* TODO : We will be moving over to
|
||||
* compare(Cell, Cell) so that the key is also converted to a cell
|
||||
*/
|
||||
public final int compare(Cell left, byte[] key, int offset, int length) {
|
||||
// row
|
||||
|
@ -132,178 +131,102 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
|
||||
/**
|
||||
* Compares the family and qualifier part of the cell
|
||||
* TODO : Handle BB cases here
|
||||
* @param left the left cell
|
||||
* @param right the right cell
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareColumns(final Cell left, final Cell right) {
|
||||
int lfoffset = left.getFamilyOffset();
|
||||
int rfoffset = right.getFamilyOffset();
|
||||
int lclength = left.getQualifierLength();
|
||||
int rclength = right.getQualifierLength();
|
||||
int lfamilylength = left.getFamilyLength();
|
||||
int rfamilylength = right.getFamilyLength();
|
||||
int diff = compareFamilies(left.getFamilyArray(), lfoffset, lfamilylength,
|
||||
right.getFamilyArray(), rfoffset, rfamilylength);
|
||||
int diff = compareFamilies(left, right);
|
||||
if (diff != 0) {
|
||||
return diff;
|
||||
} else {
|
||||
return compareQualifiers(left.getQualifierArray(), left.getQualifierOffset(), lclength,
|
||||
right.getQualifierArray(), right.getQualifierOffset(), rclength);
|
||||
}
|
||||
return compareQualifiers(left, right);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the family and qualifier part of the cell
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lfamilyOffset
|
||||
* @param lfamilylength
|
||||
* @param lqualOffset
|
||||
* @param lQualLength
|
||||
* @param right
|
||||
* @param rfamilyOffset
|
||||
* @param rfamilylength
|
||||
* @param rqualOffset
|
||||
* @param rqualLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareColumns(final Cell left, int lfamilyOffset, int lfamilylength,
|
||||
int lqualOffset, int lQualLength, final Cell right, final int rfamilyOffset,
|
||||
final int rfamilylength, final int rqualOffset, int rqualLength) {
|
||||
int diff = compareFamilies(left.getFamilyArray(), lfamilyOffset, lfamilylength,
|
||||
right.getFamilyArray(), rfamilyOffset, rfamilylength);
|
||||
if (diff != 0) {
|
||||
private final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
|
||||
int rqoffset, int rqlength) {
|
||||
int diff = compareFamilies(left, right, rfoffset, rflength);
|
||||
if (diff != 0)
|
||||
return diff;
|
||||
} else {
|
||||
return compareQualifiers(left.getQualifierArray(), lqualOffset, lQualLength,
|
||||
right.getQualifierArray(), rqualOffset, rqualLength);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the family and qualifier part of a cell with a serialized Key value byte[]
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left the cell to be compared
|
||||
* @param lfamilyOffset
|
||||
* @param lfamilylength
|
||||
* @param lqualOffset
|
||||
* @param lQualLength
|
||||
* @param right the serialized key value byte array to be compared
|
||||
* @param rfamilyOffset
|
||||
* @param rfamilylength
|
||||
* @param rqualOffset
|
||||
* @param rqualLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareColumns(final Cell left, final int lfamilyOffset,
|
||||
final int lfamilylength, final int lqualOffset, final int lQualLength, final byte[] right,
|
||||
final int rfamilyOffset, final int rfamilylength, final int rqualOffset,
|
||||
final int rqualLength) {
|
||||
int diff = compareFamilies(left.getFamilyArray(), lfamilyOffset, lfamilylength, right,
|
||||
rfamilyOffset, rfamilylength);
|
||||
if (diff != 0) {
|
||||
return diff;
|
||||
} else {
|
||||
return compareQualifiers(left.getQualifierArray(), lqualOffset, lQualLength, right,
|
||||
rqualOffset, rqualLength);
|
||||
}
|
||||
return compareQualifiers(left, right, rqoffset, rqlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare the families of left and right cell
|
||||
* TODO : Handle BB cases here
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareFamilies(Cell left, Cell right) {
|
||||
return compareFamilies(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(),
|
||||
((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), right.getFamilyLength());
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), right.getFamilyLength(),
|
||||
left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength()));
|
||||
}
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lOffset
|
||||
* @param lLength
|
||||
* @param right
|
||||
* @param rOffset
|
||||
* @param rLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareFamilies(Cell left, int lOffset, int lLength, Cell right,
|
||||
int rOffset, int rLength) {
|
||||
return compareFamilies(left.getFamilyArray(), lOffset, lLength, right.getFamilyArray(),
|
||||
rOffset, rLength);
|
||||
}
|
||||
|
||||
private final static int compareFamilies(Cell left, int lOffset, int lLength, byte[] right,
|
||||
int rOffset, int rLength) {
|
||||
return compareFamilies(left.getFamilyArray(), lOffset, lLength, right, rOffset, rLength);
|
||||
}
|
||||
|
||||
private final static int compareFamilies(byte[] leftFamily, int lFamOffset, int lFamLength,
|
||||
byte[] rightFamily, int rFamOffset, int rFamLen) {
|
||||
return Bytes.compareTo(leftFamily, lFamOffset, lFamLength, rightFamily, rFamOffset, rFamLen);
|
||||
private final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
right, roffset, rlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare the qualifiers part of the left and right cells.
|
||||
* TODO : Handle BB cases here
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareQualifiers(Cell left, Cell right) {
|
||||
return compareQualifiers(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils
|
||||
.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPositionInByteBuffer(),
|
||||
left.getQualifierLength(), ((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPositionInByteBuffer(),
|
||||
right.getQualifierLength());
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), left.getQualifierLength(),
|
||||
right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPositionInByteBuffer(),
|
||||
right.getQualifierLength(), left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength()));
|
||||
}
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
|
||||
right.getQualifierLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lOffset
|
||||
* @param lLength
|
||||
* @param right
|
||||
* @param rOffset
|
||||
* @param rLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareQualifiers(Cell left, int lOffset, int lLength, Cell right,
|
||||
int rOffset, int rLength) {
|
||||
return compareQualifiers(left.getQualifierArray(), lOffset,
|
||||
lLength, right.getQualifierArray(), rOffset,
|
||||
rLength);
|
||||
}
|
||||
|
||||
/**
|
||||
* We explicitly pass the offset and length details of the cells to avoid
|
||||
* re-parsing of the offset and length from the cell. Used only internally.
|
||||
* @param left
|
||||
* @param lOffset
|
||||
* @param lLength
|
||||
* @param right
|
||||
* @param rOffset
|
||||
* @param rLength
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
private final static int compareQualifiers(Cell left, int lOffset, int lLength, byte[] right,
|
||||
int rOffset, int rLength) {
|
||||
return compareQualifiers(left.getQualifierArray(), lOffset,
|
||||
lLength, right, rOffset,
|
||||
rLength);
|
||||
}
|
||||
|
||||
private static int compareQualifiers(byte[] leftCol, int lColOffset, int lColLength,
|
||||
byte[] rightCol, int rColOffset, int rColLength) {
|
||||
return Bytes.compareTo(leftCol, lColOffset, lColLength, rightCol, rColOffset, rColLength);
|
||||
public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), left.getQualifierLength(),
|
||||
right, rOffset, rLength);
|
||||
}
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right, rOffset, rLength);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -312,7 +235,6 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* the "same-prefix" comparator. Note that we are assuming that row portions
|
||||
* of both KVs have already been parsed and found identical, and we don't
|
||||
* validate that assumption here.
|
||||
* TODO : we will have to handle BB cases here
|
||||
* @param commonPrefix
|
||||
* the length of the common prefix of the two key-values being
|
||||
* compared, including row length and row
|
||||
|
@ -348,7 +270,6 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
return -1;
|
||||
}
|
||||
|
||||
int lfamilyoffset = left.getFamilyOffset();
|
||||
int rfamilyoffset = commonLength + roffset;
|
||||
|
||||
// Column family length.
|
||||
|
@ -359,16 +280,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
boolean sameFamilySize = (lfamilylength == rfamilylength);
|
||||
if (!sameFamilySize) {
|
||||
// comparing column family is enough.
|
||||
return compareFamilies(left, lfamilyoffset, lfamilylength, right,
|
||||
rfamilyoffset, rfamilylength);
|
||||
return compareFamilies(left, right, rfamilyoffset, rfamilylength);
|
||||
}
|
||||
// Compare family & qualifier together.
|
||||
// Families are same. Compare on qualifiers.
|
||||
int lQualOffset = left.getQualifierOffset();
|
||||
int lQualLength = left.getQualifierLength();
|
||||
int comparison = compareColumns(left, lfamilyoffset, lfamilylength, lQualOffset, lQualLength,
|
||||
right, rfamilyoffset, rfamilylength, rfamilyoffset + rfamilylength,
|
||||
(rcolumnlength - rfamilylength));
|
||||
int comparison = compareColumns(left, right, rfamilyoffset, rfamilylength, rfamilyoffset
|
||||
+ rfamilylength, (rcolumnlength - rfamilylength));
|
||||
if (comparison != 0) {
|
||||
return comparison;
|
||||
}
|
||||
|
@ -392,12 +309,27 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* Compares the rows of the left and right cell.
|
||||
* For the hbase:meta case this method is overridden such that it can handle hbase:meta cells.
|
||||
* The caller should ensure using the appropriate comparator for hbase:meta.
|
||||
* TODO : Handle BB cases here
|
||||
* @param left
|
||||
* @param right
|
||||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public int compareRows(final Cell left, final Cell right) {
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(),
|
||||
((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPositionInByteBuffer(), right.getRowLength());
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPositionInByteBuffer(), right.getRowLength(),
|
||||
left.getRowArray(), left.getRowOffset(), left.getRowLength()));
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
|
@ -418,11 +350,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return 0 if both cell and the byte[] are equal, 1 if the cell is bigger
|
||||
* than byte[], -1 otherwise
|
||||
*/
|
||||
public int compareRows(Cell left, byte[] right, int roffset,
|
||||
int rlength) {
|
||||
// TODO : for BB based cells all the hasArray based checks would happen
|
||||
// here. But we may have
|
||||
// to end up in multiple APIs accepting byte[] and BBs
|
||||
public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
|
@ -447,18 +380,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
&& right.getTypeByte() == Type.Minimum.getCode()) {
|
||||
return -1;
|
||||
}
|
||||
boolean sameFamilySize = (lFamLength == rFamLength);
|
||||
int lFamOffset = left.getFamilyOffset();
|
||||
int rFamOffset = right.getFamilyOffset();
|
||||
if (!sameFamilySize) {
|
||||
if (lFamLength != rFamLength) {
|
||||
// comparing column family is enough.
|
||||
return compareFamilies(left, lFamOffset, lFamLength, right, rFamOffset, rFamLength);
|
||||
return compareFamilies(left, right);
|
||||
}
|
||||
// Families are same. Compare on qualifiers.
|
||||
int lQualOffset = left.getQualifierOffset();
|
||||
int rQualOffset = right.getQualifierOffset();
|
||||
int diff = compareColumns(left, lFamOffset, lFamLength, lQualOffset, lQualLength, right,
|
||||
rFamOffset, rFamLength, rQualOffset, rQualLength);
|
||||
// Compare cf:qualifier
|
||||
int diff = compareColumns(left, right);
|
||||
if (diff != 0) return diff;
|
||||
|
||||
diff = compareTimestamps(left, right);
|
||||
|
@ -521,24 +448,18 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
// left is "bigger", i.e. it appears later in the sorted order
|
||||
return 1;
|
||||
}
|
||||
int qualLen = currentCell.getQualifierLength();
|
||||
if (flen + clen == 0 && type == Type.Minimum.getCode()) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
compare = compareFamilies(nextIndexedCell, nextIndexedCell.getFamilyOffset(),
|
||||
nextIndexedCell.getFamilyLength(), currentCell, currentCell.getFamilyOffset(),
|
||||
flen);
|
||||
compare = compareFamilies(nextIndexedCell, currentCell);
|
||||
if (compare != 0) {
|
||||
return compare;
|
||||
}
|
||||
if (colHint == null) {
|
||||
compare = compareQualifiers(nextIndexedCell, nextIndexedCell.getQualifierOffset(),
|
||||
nextIndexedCell.getQualifierLength(), currentCell, currentCell.getQualifierOffset(),
|
||||
qualLen);
|
||||
compare = compareQualifiers(nextIndexedCell, currentCell);
|
||||
} else {
|
||||
compare = compareQualifiers(nextIndexedCell, nextIndexedCell.getQualifierOffset(),
|
||||
nextIndexedCell.getQualifierLength(), colHint, coff, clen);
|
||||
compare = compareQualifiers(nextIndexedCell, colHint, coff, clen);
|
||||
}
|
||||
if (compare != 0) {
|
||||
return compare;
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HConstants.EMPTY_BYTE_ARRAY;
|
|||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -114,27 +115,55 @@ public final class CellUtil {
|
|||
/******************** copyTo **********************************/
|
||||
|
||||
public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
|
||||
cell.getRowLength());
|
||||
return destinationOffset + cell.getRowLength();
|
||||
short rowLen = cell.getRowLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPositionInByteBuffer(), destinationOffset, rowLen);
|
||||
} else {
|
||||
System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
|
||||
rowLen);
|
||||
}
|
||||
return destinationOffset + rowLen;
|
||||
}
|
||||
|
||||
public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset,
|
||||
cell.getFamilyLength());
|
||||
return destinationOffset + cell.getFamilyLength();
|
||||
byte fLen = cell.getFamilyLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPositionInByteBuffer(), destinationOffset, fLen);
|
||||
} else {
|
||||
System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
|
||||
destinationOffset, fLen);
|
||||
}
|
||||
return destinationOffset + fLen;
|
||||
}
|
||||
|
||||
public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
|
||||
destinationOffset, cell.getQualifierLength());
|
||||
return destinationOffset + cell.getQualifierLength();
|
||||
int qlen = cell.getQualifierLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPositionInByteBuffer(), destinationOffset, qlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
|
||||
destinationOffset, qlen);
|
||||
}
|
||||
return destinationOffset + qlen;
|
||||
}
|
||||
|
||||
public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
|
||||
cell.getValueLength());
|
||||
return destinationOffset + cell.getValueLength();
|
||||
int vlen = cell.getValueLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePositionInByteBuffer(), destinationOffset, vlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
|
||||
vlen);
|
||||
}
|
||||
return destinationOffset + vlen;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -145,14 +174,25 @@ public final class CellUtil {
|
|||
* @return position after tags
|
||||
*/
|
||||
public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
|
||||
cell.getTagsLength());
|
||||
return destinationOffset + cell.getTagsLength();
|
||||
int tlen = cell.getTagsLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferedCell) cell).getTagsPositionInByteBuffer(), destinationOffset, tlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
|
||||
tlen);
|
||||
}
|
||||
return destinationOffset + tlen;
|
||||
}
|
||||
|
||||
/********************* misc *************************************/
|
||||
|
||||
public static byte getRowByte(Cell cell, int index) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ((ByteBufferedCell) cell).getRowByteBuffer().get(
|
||||
((ByteBufferedCell) cell).getRowPositionInByteBuffer() + index);
|
||||
}
|
||||
return cell.getRowArray()[cell.getRowOffset() + index];
|
||||
}
|
||||
|
||||
|
@ -162,7 +202,14 @@ public final class CellUtil {
|
|||
return buffer;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cell
|
||||
* @return cell's qualifier wrapped into a ByteBuffer.
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
|
||||
*/
|
||||
@Deprecated
|
||||
public static ByteBuffer getQualifierBufferShallowCopy(Cell cell) {
|
||||
// No usage of this in code.
|
||||
ByteBuffer buffer = ByteBuffer.wrap(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
return buffer;
|
||||
|
@ -374,10 +421,12 @@ public final class CellUtil {
|
|||
* @param left
|
||||
* @param right
|
||||
* @return True if the rows in <code>left</code> and <code>right</code> Cells match
|
||||
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
|
||||
* Instead use {@link #matchingRows(Cell, Cell)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static boolean matchingRow(final Cell left, final Cell right) {
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
return matchingRows(left, right);
|
||||
}
|
||||
|
||||
public static boolean matchingRow(final Cell left, final byte[] buf) {
|
||||
|
@ -389,11 +438,35 @@ public final class CellUtil {
|
|||
|
||||
public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(), buf, offset,
|
||||
length) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
|
||||
length);
|
||||
}
|
||||
|
||||
public static boolean matchingFamily(final Cell left, final Cell right) {
|
||||
byte lfamlength = left.getFamilyLength();
|
||||
byte rfamlength = right.getFamilyLength();
|
||||
if (lfamlength != rfamlength) return false;
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), lfamlength,
|
||||
((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), rfamlength) == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), lfamlength,
|
||||
right.getFamilyArray(), right.getFamilyOffset(), rfamlength) == 0;
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), rfamlength,
|
||||
left.getFamilyArray(), left.getFamilyOffset(), lfamlength) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
|
@ -407,11 +480,35 @@ public final class CellUtil {
|
|||
|
||||
public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(), buf,
|
||||
offset, length) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
|
||||
offset, length);
|
||||
}
|
||||
|
||||
public static boolean matchingQualifier(final Cell left, final Cell right) {
|
||||
int lqlength = left.getQualifierLength();
|
||||
int rqlength = right.getQualifierLength();
|
||||
if (lqlength != rqlength) return false;
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), lqlength,
|
||||
((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPositionInByteBuffer(), rqlength) == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), lqlength,
|
||||
right.getQualifierArray(), right.getQualifierOffset(), rqlength) == 0;
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPositionInByteBuffer(), rqlength,
|
||||
left.getQualifierArray(), left.getQualifierOffset(), lqlength) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
|
||||
right.getQualifierLength());
|
||||
|
@ -445,6 +542,11 @@ public final class CellUtil {
|
|||
if (buf == null) {
|
||||
return left.getQualifierLength() == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), left.getQualifierLength(),
|
||||
buf, offset, length) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), buf, offset, length);
|
||||
}
|
||||
|
@ -469,11 +571,35 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
public static boolean matchingValue(final Cell left, final Cell right) {
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
|
||||
right.getValueArray(), right.getValueOffset(), right.getValueLength());
|
||||
int lvlength = left.getValueLength();
|
||||
int rvlength = right.getValueLength();
|
||||
if (lvlength != rvlength) return false;
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferedCell) left).getValuePositionInByteBuffer(), lvlength,
|
||||
((ByteBufferedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferedCell) right).getValuePositionInByteBuffer(), rvlength) == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferedCell) left).getValuePositionInByteBuffer(), lvlength,
|
||||
right.getValueArray(), right.getValueOffset(), rvlength) == 0;
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferedCell) right).getValuePositionInByteBuffer(), rvlength,
|
||||
left.getValueArray(), left.getValueOffset(), lvlength) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
|
||||
right.getValueArray(), right.getValueOffset(), rvlength);
|
||||
}
|
||||
|
||||
public static boolean matchingValue(final Cell left, final byte[] buf) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferedCell) left).getValuePositionInByteBuffer(), left.getValueLength(), buf, 0,
|
||||
buf.length) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
|
||||
buf.length);
|
||||
}
|
||||
|
@ -967,68 +1093,102 @@ public final class CellUtil {
|
|||
public static boolean matchingRows(final Cell left, final Cell right) {
|
||||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
return matchingRows(left, lrowlength, right, rrowlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param left
|
||||
* @param lrowlength
|
||||
* @param right
|
||||
* @param rrowlength
|
||||
* @return True if rows match.
|
||||
*/
|
||||
private static boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
|
||||
final short rrowlength) {
|
||||
return lrowlength == rrowlength
|
||||
&& matchingRows(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
|
||||
right.getRowOffset(), rrowlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare rows. Just calls Bytes.equals, but it's good to have this
|
||||
* encapsulated.
|
||||
*
|
||||
* @param left
|
||||
* Left row array.
|
||||
* @param loffset
|
||||
* Left row offset.
|
||||
* @param llength
|
||||
* Left row length.
|
||||
* @param right
|
||||
* Right row array.
|
||||
* @param roffset
|
||||
* Right row offset.
|
||||
* @param rlength
|
||||
* Right row length.
|
||||
* @return Whether rows are the same row.
|
||||
*/
|
||||
private static boolean matchingRows(final byte[] left, final int loffset, final int llength,
|
||||
final byte[] right, final int roffset, final int rlength) {
|
||||
return Bytes.equals(left, loffset, llength, right, roffset, rlength);
|
||||
if (lrowlength != rrowlength) return false;
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPositionInByteBuffer(), lrowlength,
|
||||
((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPositionInByteBuffer(), rrowlength) == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPositionInByteBuffer(), lrowlength, right.getRowArray(),
|
||||
right.getRowOffset(), rrowlength) == 0;
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPositionInByteBuffer(), rrowlength, left.getRowArray(),
|
||||
left.getRowOffset(), lrowlength) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the row and column of two keyvalues for equality
|
||||
*
|
||||
*
|
||||
* @param left
|
||||
* @param right
|
||||
* @return True if same row and column.
|
||||
*/
|
||||
public static boolean matchingRowColumn(final Cell left, final Cell right) {
|
||||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
|
||||
if ((lrowlength + left.getFamilyLength() + left.getQualifierLength()) != (rrowlength
|
||||
+ right.getFamilyLength() + right.getQualifierLength())) {
|
||||
if ((left.getRowLength() + left.getFamilyLength() + left.getQualifierLength()) != (right
|
||||
.getRowLength() + right.getFamilyLength() + right.getQualifierLength())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!matchingRows(left, lrowlength, right, rrowlength)) {
|
||||
if (!matchingRows(left, right)) {
|
||||
return false;
|
||||
}
|
||||
return matchingColumn(left, right);
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the rowkey bytes of the given cell into an int value
|
||||
*
|
||||
* @param cell
|
||||
* @return rowkey as int
|
||||
*/
|
||||
public static int getRowAsInt(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPositionInByteBuffer());
|
||||
}
|
||||
return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the value bytes of the given cell into a long value
|
||||
*
|
||||
* @param cell
|
||||
* @return value as long
|
||||
*/
|
||||
public static long getValueAsLong(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toLong(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePositionInByteBuffer());
|
||||
}
|
||||
return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the value bytes of the given cell into a double value
|
||||
*
|
||||
* @param cell
|
||||
* @return value as double
|
||||
*/
|
||||
public static double getValueAsDouble(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toDouble(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePositionInByteBuffer());
|
||||
}
|
||||
return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the value bytes of the given cell into a BigDecimal
|
||||
*
|
||||
* @param cell
|
||||
* @return value as BigDecimal
|
||||
*/
|
||||
public static BigDecimal getValueAsBigDecimal(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toBigDecimal(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePositionInByteBuffer(), cell.getValueLength());
|
||||
}
|
||||
return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Cell that is smaller than all other possible Cells for the given Cell's row.
|
||||
*
|
||||
|
@ -1039,6 +1199,71 @@ public final class CellUtil {
|
|||
return new FirstOnRowFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Cell that is smaller than all other possible Cells for the given Cell row's next row.
|
||||
* Makes the next row's rowkey by appending single byte 0x00 to the end of current row key.
|
||||
*/
|
||||
public static Cell createFirstOnNextRow(final Cell cell) {
|
||||
byte[] nextRow = new byte[cell.getRowLength() + 1];
|
||||
copyRowTo(cell, nextRow, 0);
|
||||
nextRow[nextRow.length - 1] = 0;// maybe not necessary
|
||||
return new FirstOnRowFakeCell(nextRow, 0, (short) nextRow.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Cell that is smaller than all other possible Cells for the given Cell's rk:cf and
|
||||
* passed qualifier.
|
||||
*
|
||||
* @param cell
|
||||
* @param qArray
|
||||
* @param qoffest
|
||||
* @param qlength
|
||||
* @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
|
||||
*/
|
||||
public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
|
||||
return new FirstOnRowColumnFakeCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
qArray, qoffest, qlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the first cell with the row/family/qualifier of this cell and the given timestamp.
|
||||
* Uses the "maximum" type that guarantees that the new cell is the lowest possible for this
|
||||
* combination of row, family, qualifier, and timestamp. This cell's own timestamp is ignored.
|
||||
*
|
||||
* @param cell - cell
|
||||
* @param ts
|
||||
*/
|
||||
public static Cell createFirstOnRowColTS(Cell cell, long ts) {
|
||||
return new FirstOnRowColumnTSFakeCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), ts);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Cell that is larger than all other possible Cells for the given Cell's row.
|
||||
*
|
||||
* @param cell
|
||||
* @return Last possible Cell on passed Cell's row.
|
||||
*/
|
||||
public static Cell createLastOnRow(final Cell cell) {
|
||||
return new LastOnRowFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a Cell that is larger than all other possible Cells for the given Cell's rk:cf:q. Used
|
||||
* in creating "fake keys" for the multi-column Bloom filter optimization to skip the row/column
|
||||
* we already know is not in the file.
|
||||
*
|
||||
* @param cell
|
||||
* @return Last possible Cell on passed Cell's rk:cf:q.
|
||||
*/
|
||||
public static Cell createLastOnRowCol(final Cell cell) {
|
||||
return new LastOnRowColumnFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static abstract class FakeCell implements Cell {
|
||||
|
||||
|
@ -1151,12 +1376,12 @@ public final class CellUtil {
|
|||
@InterfaceAudience.Private
|
||||
private static class FirstOnRowFakeCell extends FakeCell {
|
||||
private final byte[] rowArray;
|
||||
private final int roffest;
|
||||
private final int roffset;
|
||||
private final short rlength;
|
||||
|
||||
public FirstOnRowFakeCell(final byte[] row, int roffset, short rlength) {
|
||||
this.rowArray = row;
|
||||
this.roffest = roffset;
|
||||
this.roffset = roffset;
|
||||
this.rlength = rlength;
|
||||
}
|
||||
|
||||
|
@ -1167,7 +1392,7 @@ public final class CellUtil {
|
|||
|
||||
@Override
|
||||
public int getRowOffset() {
|
||||
return this.roffest;
|
||||
return this.roffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1187,7 +1412,189 @@ public final class CellUtil {
|
|||
|
||||
@Override
|
||||
public byte[] getRow() {
|
||||
return Bytes.copy(this.rowArray, this.roffest, this.rlength);
|
||||
return Bytes.copy(this.rowArray, this.roffset, this.rlength);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class FirstOnRowColumnFakeCell extends FirstOnRowFakeCell {
|
||||
private final byte[] fArray;
|
||||
private final int foffset;
|
||||
private final byte flength;
|
||||
private final byte[] qArray;
|
||||
private final int qoffset;
|
||||
private final int qlength;
|
||||
|
||||
public FirstOnRowColumnFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
|
||||
int foffset, byte flength, byte[] qArray, int qoffset, int qlength) {
|
||||
super(rArray, roffset, rlength);
|
||||
this.fArray = fArray;
|
||||
this.foffset = foffset;
|
||||
this.flength = flength;
|
||||
this.qArray = qArray;
|
||||
this.qoffset = qoffset;
|
||||
this.qlength = qlength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getFamilyArray() {
|
||||
return this.fArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getFamilyOffset() {
|
||||
return this.foffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getFamilyLength() {
|
||||
return this.flength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getQualifierArray() {
|
||||
return this.qArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getQualifierOffset() {
|
||||
return this.qoffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getQualifierLength() {
|
||||
return this.qlength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getFamily() {
|
||||
return Bytes.copy(this.fArray, this.foffset, this.flength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getQualifier() {
|
||||
return Bytes.copy(this.qArray, this.qoffset, this.qlength);
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class FirstOnRowColumnTSFakeCell extends FirstOnRowColumnFakeCell {
|
||||
|
||||
private long ts;
|
||||
|
||||
public FirstOnRowColumnTSFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
|
||||
int foffset, byte flength, byte[] qArray, int qoffset, int qlength, long ts) {
|
||||
super(rArray, roffset, rlength, fArray, foffset, flength, qArray, qoffset, qlength);
|
||||
this.ts = ts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestamp() {
|
||||
return this.ts;
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class LastOnRowFakeCell extends FakeCell {
|
||||
private final byte[] rowArray;
|
||||
private final int roffset;
|
||||
private final short rlength;
|
||||
|
||||
public LastOnRowFakeCell(byte[] row, int roffset, short rlength) {
|
||||
this.rowArray = row;
|
||||
this.roffset = roffset;
|
||||
this.rlength = rlength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getRowArray() {
|
||||
return this.rowArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getRowOffset() {
|
||||
return this.roffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getRowLength() {
|
||||
return this.rlength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestamp() {
|
||||
return HConstants.OLDEST_TIMESTAMP;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getTypeByte() {
|
||||
return Type.Minimum.getCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getRow() {
|
||||
return Bytes.copy(this.rowArray, this.roffset, this.rlength);
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class LastOnRowColumnFakeCell extends LastOnRowFakeCell {
|
||||
private final byte[] fArray;
|
||||
private final int foffset;
|
||||
private final byte flength;
|
||||
private final byte[] qArray;
|
||||
private final int qoffset;
|
||||
private final int qlength;
|
||||
|
||||
public LastOnRowColumnFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
|
||||
int foffset, byte flength, byte[] qArray, int qoffset, int qlength) {
|
||||
super(rArray, roffset, rlength);
|
||||
this.fArray = fArray;
|
||||
this.foffset = foffset;
|
||||
this.flength = flength;
|
||||
this.qArray = qArray;
|
||||
this.qoffset = qoffset;
|
||||
this.qlength = qlength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getFamilyArray() {
|
||||
return this.fArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getFamilyOffset() {
|
||||
return this.foffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getFamilyLength() {
|
||||
return this.flength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getQualifierArray() {
|
||||
return this.qArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getQualifierOffset() {
|
||||
return this.qoffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getQualifierLength() {
|
||||
return this.qlength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getFamily() {
|
||||
return Bytes.copy(this.fArray, this.foffset, this.flength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getQualifier() {
|
||||
return Bytes.copy(this.qArray, this.qoffset, this.qlength);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.io.util.StreamUtils;
|
|||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.IterableUtils;
|
||||
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
|
@ -221,26 +220,6 @@ public class KeyValueUtil {
|
|||
|
||||
/*************** next/previous **********************************/
|
||||
|
||||
/**
|
||||
* Append single byte 0x00 to the end of the input row key
|
||||
*/
|
||||
public static KeyValue createFirstKeyInNextRow(final Cell in){
|
||||
byte[] nextRow = new byte[in.getRowLength() + 1];
|
||||
System.arraycopy(in.getRowArray(), in.getRowOffset(), nextRow, 0, in.getRowLength());
|
||||
nextRow[nextRow.length - 1] = 0;//maybe not necessary
|
||||
return createFirstOnRow(nextRow);
|
||||
}
|
||||
|
||||
/**
|
||||
* Increment the row bytes and clear the other fields
|
||||
*/
|
||||
public static KeyValue createFirstKeyInIncrementedRow(final Cell in){
|
||||
byte[] thisRow = new SimpleMutableByteRange(in.getRowArray(), in.getRowOffset(),
|
||||
in.getRowLength()).deepCopyToNewArray();
|
||||
byte[] nextRow = Bytes.unsignedCopyAndIncrement(thisRow);
|
||||
return createFirstOnRow(nextRow);
|
||||
}
|
||||
|
||||
/**
|
||||
* Decrement the timestamp. For tests (currently wasteful)
|
||||
*
|
||||
|
@ -285,53 +264,7 @@ public class KeyValueUtil {
|
|||
return new KeyValue(row, roffset, rlength, family, foffset, flength, qualifier, qoffset,
|
||||
qlength, HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a keyValue for the specified keyvalue larger than or equal to all other possible
|
||||
* KeyValues that have the same row, family, qualifer. Used for reseeking
|
||||
* @param kv
|
||||
* @return KeyValue
|
||||
*/
|
||||
public static KeyValue createLastOnRow(Cell kv) {
|
||||
return createLastOnRow(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), null, 0, 0,
|
||||
null, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Similar to
|
||||
* {@link #createLastOnRow(byte[], int, int, byte[], int, int, byte[], int, int)}
|
||||
* but creates the last key on the row/column of this KV (the value part of
|
||||
* the returned KV is always empty). Used in creating "fake keys" for the
|
||||
* multi-column Bloom filter optimization to skip the row/column we already
|
||||
* know is not in the file.
|
||||
*
|
||||
* @param kv - cell
|
||||
* @return the last key on the row/column of the given key-value pair
|
||||
*/
|
||||
public static KeyValue createLastOnRowCol(Cell kv) {
|
||||
return new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(),
|
||||
kv.getQualifierOffset(), kv.getQualifierLength(), HConstants.OLDEST_TIMESTAMP,
|
||||
Type.Minimum, null, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the first KV with the row/family/qualifier of this KV and the given
|
||||
* timestamp. Uses the "maximum" KV type that guarantees that the new KV is
|
||||
* the lowest possible for this combination of row, family, qualifier, and
|
||||
* timestamp. This KV's own timestamp is ignored. While this function copies
|
||||
* the value from this KV, it is normally used on key-only KVs.
|
||||
*
|
||||
* @param kv - cell
|
||||
* @param ts
|
||||
*/
|
||||
public static KeyValue createFirstOnRowColTS(Cell kv, long ts) {
|
||||
return new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(),
|
||||
kv.getQualifierOffset(), kv.getQualifierLength(), ts, Type.Maximum, kv.getValueArray(),
|
||||
kv.getValueOffset(), kv.getValueLength());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a KeyValue that is smaller than all other possible KeyValues
|
||||
* for the given row. That is any (valid) KeyValue on 'row' would sort
|
||||
|
@ -344,7 +277,6 @@ public class KeyValueUtil {
|
|||
return new KeyValue(row, roffset, rlength,
|
||||
null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a KeyValue that is last on the specified row id. That is,
|
||||
|
@ -516,21 +448,6 @@ public class KeyValueUtil {
|
|||
return new KeyValue(buffer, boffset, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the first KV with the row/family/qualifier of this KV and the
|
||||
* given timestamp. Uses the "maximum" KV type that guarantees that the new
|
||||
* KV is the lowest possible for this combination of row, family, qualifier,
|
||||
* and timestamp. This KV's own timestamp is ignored. While this function
|
||||
* copies the value from this KV, it is normally used on key-only KVs.
|
||||
*/
|
||||
public static KeyValue createFirstOnRowColTS(KeyValue kv, long ts) {
|
||||
return new KeyValue(
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
|
||||
ts, Type.Maximum, kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
}
|
||||
|
||||
/*************** misc **********************************/
|
||||
/**
|
||||
* @param cell
|
||||
|
|
|
@ -68,6 +68,7 @@ public class CellCodec implements Codec {
|
|||
*/
|
||||
private void write(final byte [] bytes, final int offset, final int length)
|
||||
throws IOException {
|
||||
// TODO add BB backed os check and do for write. Pass Cell
|
||||
this.out.write(Bytes.toBytes(length));
|
||||
this.out.write(bytes, offset, length);
|
||||
}
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.io.DataInputStream;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.BigInteger;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -28,6 +30,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import sun.nio.ch.DirectBuffer;
|
||||
|
||||
/**
|
||||
* Utility functions for working with byte buffers, such as reading/writing
|
||||
* variable-length long numbers.
|
||||
|
@ -511,7 +515,21 @@ public final class ByteBufferUtils {
|
|||
|
||||
public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
|
||||
if (UnsafeAccess.isAvailable()) {
|
||||
return compareToUnsafe(buf1, o1, l1, buf2, o2, l2);
|
||||
long offset1Adj, offset2Adj;
|
||||
Object refObj1 = null, refObj2 = null;
|
||||
if (buf1.hasArray()) {
|
||||
offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
|
||||
refObj1 = buf1.array();
|
||||
} else {
|
||||
offset1Adj = o1 + ((DirectBuffer) buf1).address();
|
||||
}
|
||||
if (buf2.hasArray()) {
|
||||
offset2Adj = o2 + buf2.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
|
||||
refObj2 = buf2.array();
|
||||
} else {
|
||||
offset2Adj = o2 + ((DirectBuffer) buf2).address();
|
||||
}
|
||||
return compareToUnsafe(refObj1, offset1Adj, l1, refObj2, offset2Adj, l2);
|
||||
}
|
||||
int end1 = o1 + l1;
|
||||
int end2 = o2 + l2;
|
||||
|
@ -525,7 +543,32 @@ public final class ByteBufferUtils {
|
|||
return l1 - l2;
|
||||
}
|
||||
|
||||
static int compareToUnsafe(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
|
||||
public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
|
||||
if (UnsafeAccess.isAvailable()) {
|
||||
long offset1Adj;
|
||||
Object refObj1 = null;
|
||||
if (buf1.hasArray()) {
|
||||
offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
|
||||
refObj1 = buf1.array();
|
||||
} else {
|
||||
offset1Adj = o1 + ((DirectBuffer) buf1).address();
|
||||
}
|
||||
return compareToUnsafe(refObj1, offset1Adj, l1, buf2, o2
|
||||
+ UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l2);
|
||||
}
|
||||
int end1 = o1 + l1;
|
||||
int end2 = o2 + l2;
|
||||
for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
|
||||
int a = buf1.get(i) & 0xFF;
|
||||
int b = buf2[j] & 0xFF;
|
||||
if (a != b) {
|
||||
return a - b;
|
||||
}
|
||||
}
|
||||
return l1 - l2;
|
||||
}
|
||||
|
||||
static int compareToUnsafe(Object obj1, long o1, int l1, Object obj2, long o2, int l2) {
|
||||
final int minLength = Math.min(l1, l2);
|
||||
final int minWords = minLength / Bytes.SIZEOF_LONG;
|
||||
|
||||
|
@ -536,8 +579,8 @@ public final class ByteBufferUtils {
|
|||
*/
|
||||
int j = minWords << 3; // Same as minWords * SIZEOF_LONG
|
||||
for (int i = 0; i < j; i += Bytes.SIZEOF_LONG) {
|
||||
long lw = UnsafeAccess.getAsLong(buf1, o1 + i);
|
||||
long rw = UnsafeAccess.getAsLong(buf2, o2 + i);
|
||||
long lw = UnsafeAccess.theUnsafe.getLong(obj1, o1 + i);
|
||||
long rw = UnsafeAccess.theUnsafe.getLong(obj2, o2 + i);
|
||||
long diff = lw ^ rw;
|
||||
if (diff != 0) {
|
||||
return lessThanUnsignedLong(lw, rw) ? -1 : 1;
|
||||
|
@ -546,24 +589,24 @@ public final class ByteBufferUtils {
|
|||
int offset = j;
|
||||
|
||||
if (minLength - offset >= Bytes.SIZEOF_INT) {
|
||||
int il = UnsafeAccess.getAsInt(buf1, o1 + offset);
|
||||
int ir = UnsafeAccess.getAsInt(buf2, o2 + offset);
|
||||
int il = UnsafeAccess.theUnsafe.getInt(obj1, o1 + offset);
|
||||
int ir = UnsafeAccess.theUnsafe.getInt(obj2, o2 + offset);
|
||||
if (il != ir) {
|
||||
return lessThanUnsignedInt(il, ir) ? -1 : 1;
|
||||
}
|
||||
offset += Bytes.SIZEOF_INT;
|
||||
}
|
||||
if (minLength - offset >= Bytes.SIZEOF_SHORT) {
|
||||
short sl = UnsafeAccess.getAsShort(buf1, o1 + offset);
|
||||
short sr = UnsafeAccess.getAsShort(buf2, o2 + offset);
|
||||
short sl = UnsafeAccess.theUnsafe.getShort(obj1, o1 + offset);
|
||||
short sr = UnsafeAccess.theUnsafe.getShort(obj2, o2 + offset);
|
||||
if (sl != sr) {
|
||||
return lessThanUnsignedShort(sl, sr) ? -1 : 1;
|
||||
}
|
||||
offset += Bytes.SIZEOF_SHORT;
|
||||
}
|
||||
if (minLength - offset == 1) {
|
||||
int a = (buf1.get(o1 + offset) & 0xff);
|
||||
int b = (buf2.get(o2 + offset) & 0xff);
|
||||
int a = (UnsafeAccess.theUnsafe.getByte(obj1, o1 + offset) & 0xff);
|
||||
int b = (UnsafeAccess.theUnsafe.getByte(obj2, o2 + offset) & 0xff);
|
||||
if (a != b) {
|
||||
return a - b;
|
||||
}
|
||||
|
@ -664,6 +707,34 @@ public final class ByteBufferUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a double value at the given buffer's offset.
|
||||
* @param buffer
|
||||
* @param offset offset where double is
|
||||
* @return double value at offset
|
||||
*/
|
||||
public static double toDouble(ByteBuffer buffer, int offset) {
|
||||
return Double.longBitsToDouble(toLong(buffer, offset));
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a BigDecimal value at the given buffer's offset.
|
||||
* @param buffer
|
||||
* @param offset
|
||||
* @return BigDecimal value at offset
|
||||
*/
|
||||
public static BigDecimal toBigDecimal(ByteBuffer buffer, int offset, int length) {
|
||||
if (buffer == null || length < Bytes.SIZEOF_INT + 1 ||
|
||||
(offset + length > buffer.limit())) {
|
||||
return null;
|
||||
}
|
||||
|
||||
int scale = toInt(buffer, offset);
|
||||
byte[] tcBytes = new byte[length - Bytes.SIZEOF_INT];
|
||||
copyFromBufferToArray(tcBytes, buffer, offset + Bytes.SIZEOF_INT, 0, length - Bytes.SIZEOF_INT);
|
||||
return new BigDecimal(new BigInteger(tcBytes), scale);
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes
|
||||
* to buffer's current position. This also advances the position in the 'out' buffer by 'length'
|
||||
|
|
|
@ -17,9 +17,13 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -87,4 +91,31 @@ public class TestCellComparator {
|
|||
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
|
||||
assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) == 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompareByteBufferedCell() {
|
||||
byte[] r1 = Bytes.toBytes("row1");
|
||||
byte[] r2 = Bytes.toBytes("row2");
|
||||
byte[] f1 = Bytes.toBytes("cf1");
|
||||
byte[] q1 = Bytes.toBytes("qual1");
|
||||
byte[] q2 = Bytes.toBytes("qual2");
|
||||
byte[] v = Bytes.toBytes("val1");
|
||||
KeyValue kv = new KeyValue(r1, f1, q1, v);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
kv = new KeyValue(r2, f1, q1, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellComparator.compareColumns(bbCell1, bbCell2));
|
||||
assertEquals(0, CellComparator.compareColumns(bbCell1, kv));
|
||||
kv = new KeyValue(r2, f1, q2, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell3 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellComparator.compareFamilies(bbCell2, bbCell3));
|
||||
assertTrue(CellComparator.compareQualifiers(bbCell2, bbCell3) < 0);
|
||||
assertTrue(CellComparator.compareColumns(bbCell2, bbCell3) < 0);
|
||||
|
||||
assertEquals(0, CellComparator.COMPARATOR.compareRows(bbCell2, bbCell3));
|
||||
assertTrue(CellComparator.COMPARATOR.compareRows(bbCell1, bbCell2) < 0);
|
||||
}
|
||||
}
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.hbase;
|
|||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
|
@ -29,6 +31,7 @@ import java.util.TreeMap;
|
|||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -431,4 +434,283 @@ public class TestCellUtil {
|
|||
|
||||
// TODO: test with tags
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloneCellFieldsFromByteBufferedCell() {
|
||||
byte[] r = Bytes.toBytes("row1");
|
||||
byte[] f = Bytes.toBytes("cf1");
|
||||
byte[] q = Bytes.toBytes("qual1");
|
||||
byte[] v = Bytes.toBytes("val1");
|
||||
byte[] tags = Bytes.toBytes("tag1");
|
||||
KeyValue kv = new KeyValue(r, f, q, 0, q.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
byte[] rDest = CellUtil.cloneRow(bbCell);
|
||||
assertTrue(Bytes.equals(r, rDest));
|
||||
byte[] fDest = CellUtil.cloneFamily(bbCell);
|
||||
assertTrue(Bytes.equals(f, fDest));
|
||||
byte[] qDest = CellUtil.cloneQualifier(bbCell);
|
||||
assertTrue(Bytes.equals(q, qDest));
|
||||
byte[] vDest = CellUtil.cloneValue(bbCell);
|
||||
assertTrue(Bytes.equals(v, vDest));
|
||||
byte[] tDest = new byte[tags.length];
|
||||
CellUtil.copyTagTo(bbCell, tDest, 0);
|
||||
assertTrue(Bytes.equals(tags, tDest));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchingCellFieldsFromByteBufferedCell() {
|
||||
byte[] r = Bytes.toBytes("row1");
|
||||
byte[] f = Bytes.toBytes("cf1");
|
||||
byte[] q1 = Bytes.toBytes("qual1");
|
||||
byte[] q2 = Bytes.toBytes("qual2");
|
||||
byte[] v = Bytes.toBytes("val1");
|
||||
byte[] tags = Bytes.toBytes("tag1");
|
||||
KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
kv = new KeyValue(r, f, q2, 0, q2.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingRows(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingRow(bbCell1, r));
|
||||
assertTrue(CellUtil.matchingFamily(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingFamily(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingFamily(bbCell1, f));
|
||||
assertFalse(CellUtil.matchingQualifier(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingQualifier(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingQualifier(bbCell1, q1));
|
||||
assertTrue(CellUtil.matchingQualifier(bbCell2, q2));
|
||||
assertTrue(CellUtil.matchingValue(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingValue(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingValue(bbCell1, v));
|
||||
assertFalse(CellUtil.matchingColumn(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingColumn(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingColumn(bbCell1, f, q1));
|
||||
assertTrue(CellUtil.matchingColumn(bbCell2, f, q2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCellFieldsAsPrimitiveTypesFromByteBufferedCell() {
|
||||
int ri = 123;
|
||||
byte[] r = Bytes.toBytes(ri);
|
||||
byte[] f = Bytes.toBytes("cf1");
|
||||
byte[] q = Bytes.toBytes("qual1");
|
||||
long vl = 10981L;
|
||||
byte[] v = Bytes.toBytes(vl);
|
||||
KeyValue kv = new KeyValue(r, f, q, v);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(ri, CellUtil.getRowAsInt(bbCell));
|
||||
assertEquals(vl, CellUtil.getValueAsLong(bbCell));
|
||||
double vd = 3005.5;
|
||||
v = Bytes.toBytes(vd);
|
||||
kv = new KeyValue(r, f, q, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(vd, CellUtil.getValueAsDouble(bbCell), 0.0);
|
||||
BigDecimal bd = new BigDecimal(9999);
|
||||
v = Bytes.toBytes(bd);
|
||||
kv = new KeyValue(r, f, q, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
|
||||
}
|
||||
|
||||
// TODO remove this test impl once we have a Cell implementation backed by ByteBuffer
|
||||
public static class ByteBufferedCellImpl extends ByteBufferedCell {
|
||||
|
||||
private final ByteBuffer buffer;
|
||||
private final int offset, length;
|
||||
|
||||
public ByteBufferedCellImpl(ByteBuffer buffer, int offset, int length) {
|
||||
this.buffer = buffer;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getRowArray() {
|
||||
return CellUtil.cloneRow(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getRowOffset() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getRowLength() {
|
||||
return ByteBufferUtils.toShort(this.buffer, this.offset + KeyValue.ROW_OFFSET);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getFamilyArray() {
|
||||
return CellUtil.cloneFamily(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getFamilyOffset() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getFamilyLength() {
|
||||
return this.buffer.get(getFamilyPositionInByteBuffer() - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getQualifierArray() {
|
||||
return CellUtil.cloneQualifier(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getQualifierOffset() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getQualifierLength() {
|
||||
return getKeyLength()
|
||||
- (int) KeyValue.getKeyDataStructureSize(getRowLength(), getFamilyLength(), 0);
|
||||
}
|
||||
|
||||
private int getKeyLength() {
|
||||
return ByteBufferUtils.toInt(this.buffer, this.offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestamp() {
|
||||
int tsOffset = this.offset + KeyValue.ROW_OFFSET + getKeyLength()
|
||||
- KeyValue.TIMESTAMP_TYPE_SIZE;
|
||||
return ByteBufferUtils.toLong(buffer, tsOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getTypeByte() {
|
||||
return KeyValue.Type.Put.getCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMvccVersion() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSequenceId() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getValueArray() {
|
||||
return CellUtil.cloneValue(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueOffset() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueLength() {
|
||||
return ByteBufferUtils.toInt(this.buffer, this.offset + KeyValue.KEY_LENGTH_SIZE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getTagsArray() {
|
||||
byte[] tDest = new byte[getTagsLength()];
|
||||
CellUtil.copyTagTo(this, tDest, 0);
|
||||
return tDest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTagsOffset() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTagsLength() {
|
||||
int tagsLen = this.length
|
||||
- (getKeyLength() + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
|
||||
if (tagsLen > 0) {
|
||||
tagsLen -= KeyValue.TAGS_LENGTH_SIZE;
|
||||
}
|
||||
return tagsLen;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getValue() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getFamily() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getQualifier() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getRow() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
ByteBuffer getRowByteBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getRowPositionInByteBuffer() {
|
||||
return this.offset + KeyValue.ROW_KEY_OFFSET;
|
||||
}
|
||||
|
||||
@Override
|
||||
ByteBuffer getFamilyByteBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getFamilyPositionInByteBuffer() {
|
||||
return this.offset + KeyValue.ROW_KEY_OFFSET + getRowLength() + Bytes.SIZEOF_BYTE;
|
||||
}
|
||||
|
||||
@Override
|
||||
ByteBuffer getQualifierByteBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getQualifierPositionInByteBuffer() {
|
||||
return getFamilyPositionInByteBuffer() + getFamilyLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
ByteBuffer getValueByteBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getValuePositionInByteBuffer() {
|
||||
return this.offset + KeyValue.ROW_OFFSET + getKeyLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
ByteBuffer getTagsByteBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getTagsPositionInByteBuffer() {
|
||||
int tagsLen = getTagsLength();
|
||||
if (tagsLen == 0) {
|
||||
return this.offset + this.length;
|
||||
}
|
||||
return this.offset + this.length - tagsLen;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -144,7 +144,7 @@ public class TestPrefixTreeSearcher {
|
|||
KeyValue kv = rows.getInputs().get(i);
|
||||
|
||||
//nextRow
|
||||
KeyValue inputNextRow = KeyValueUtil.createFirstKeyInNextRow(kv);
|
||||
Cell inputNextRow = CellUtil.createFirstOnNextRow(kv);
|
||||
|
||||
CellScannerPosition position = beforeVsAfterOnMiss
|
||||
? searcher.positionAtOrBefore(inputNextRow)
|
||||
|
|
|
@ -439,8 +439,6 @@ public abstract class BaseRegionObserver implements RegionObserver {
|
|||
@Override
|
||||
public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
|
||||
final InternalScanner s, final Cell curRowCell, final boolean hasMore) throws IOException {
|
||||
// TODO when cell is backed by DirectByteBuffer, we would need to copy row bytes to temp byte[]
|
||||
// and call old method for BC.
|
||||
return postScannerFilterRow(e, s, curRowCell.getRowArray(), curRowCell.getRowOffset(),
|
||||
curRowCell.getRowLength(), hasMore);
|
||||
}
|
||||
|
|
|
@ -664,12 +664,8 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
*/
|
||||
@Override
|
||||
public void append(final Cell cell) throws IOException {
|
||||
byte[] value = cell.getValueArray();
|
||||
int voffset = cell.getValueOffset();
|
||||
int vlength = cell.getValueLength();
|
||||
// checkKey uses comparator to check we are writing in order.
|
||||
boolean dupKey = checkKey(cell);
|
||||
checkValue(value, voffset, vlength);
|
||||
if (!dupKey) {
|
||||
checkBlockBoundary();
|
||||
}
|
||||
|
@ -681,7 +677,7 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
fsBlockWriter.write(cell);
|
||||
|
||||
totalKeyLength += CellUtil.estimatedSerializedSizeOfKey(cell);
|
||||
totalValueLength += vlength;
|
||||
totalValueLength += cell.getValueLength();
|
||||
|
||||
// Are we the first key in this block?
|
||||
if (firstCellInBlock == null) {
|
||||
|
|
|
@ -971,8 +971,7 @@ public class DefaultMemStore implements MemStore {
|
|||
*/
|
||||
@Override
|
||||
public synchronized boolean seekToPreviousRow(Cell key) {
|
||||
Cell firstKeyOnRow = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
|
||||
key.getRowLength());
|
||||
Cell firstKeyOnRow = CellUtil.createFirstOnRow(key);
|
||||
SortedSet<Cell> cellHead = cellSetAtCreation.headSet(firstKeyOnRow);
|
||||
Cell cellSetBeforeRow = cellHead.isEmpty() ? null : cellHead.last();
|
||||
SortedSet<Cell> snapshotHead = snapshotAtCreation
|
||||
|
@ -984,8 +983,7 @@ public class DefaultMemStore implements MemStore {
|
|||
theNext = null;
|
||||
return false;
|
||||
}
|
||||
Cell firstKeyOnPreviousRow = KeyValueUtil.createFirstOnRow(lastCellBeforeRow.getRowArray(),
|
||||
lastCellBeforeRow.getRowOffset(), lastCellBeforeRow.getRowLength());
|
||||
Cell firstKeyOnPreviousRow = CellUtil.createFirstOnRow(lastCellBeforeRow);
|
||||
this.stopSkippingCellsIfNextRow = true;
|
||||
seek(firstKeyOnPreviousRow);
|
||||
this.stopSkippingCellsIfNextRow = false;
|
||||
|
@ -1006,8 +1004,7 @@ public class DefaultMemStore implements MemStore {
|
|||
if (higherCell == null) {
|
||||
return false;
|
||||
}
|
||||
Cell firstCellOnLastRow = KeyValueUtil.createFirstOnRow(higherCell.getRowArray(),
|
||||
higherCell.getRowOffset(), higherCell.getRowLength());
|
||||
Cell firstCellOnLastRow = CellUtil.createFirstOnRow(higherCell);
|
||||
if (seek(firstCellOnLastRow)) {
|
||||
return true;
|
||||
} else {
|
||||
|
|
|
@ -23,10 +23,10 @@ import java.util.NavigableSet;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
* This class is used for the tracking and enforcement of columns and numbers
|
||||
|
@ -123,19 +123,18 @@ public class ExplicitColumnTracker implements ColumnTracker {
|
|||
}
|
||||
|
||||
// Compare specific column to current column
|
||||
// TODO when cell is offheap backed, we won't use getQualifierArray()
|
||||
int ret = Bytes.compareTo(column.getBuffer(), column.getOffset(), column.getLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
|
||||
int ret = CellComparator.compareQualifiers(cell, column.getBuffer(), column.getOffset(),
|
||||
column.getLength());
|
||||
|
||||
// Column Matches. Return include code. The caller would call checkVersions
|
||||
// to limit the number of versions.
|
||||
if(ret == 0) {
|
||||
if (ret == 0) {
|
||||
return ScanQueryMatcher.MatchCode.INCLUDE;
|
||||
}
|
||||
|
||||
resetTS();
|
||||
|
||||
if (ret > 0) {
|
||||
if (ret < 0) {
|
||||
// The current KV is smaller than the column the ExplicitColumnTracker
|
||||
// is interested in, so seek to that column of interest.
|
||||
return ScanQueryMatcher.MatchCode.SEEK_NEXT_COL;
|
||||
|
@ -145,7 +144,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
|
|||
// is interested in. That means there is no more data for the column
|
||||
// of interest. Advance the ExplicitColumnTracker state to next
|
||||
// column of interest, and check again.
|
||||
if (ret <= -1) {
|
||||
if (ret > 0) {
|
||||
++this.index;
|
||||
if (done()) {
|
||||
// No more to match, do not include, done with this row.
|
||||
|
@ -221,10 +220,10 @@ public class ExplicitColumnTracker implements ColumnTracker {
|
|||
*/
|
||||
public void doneWithColumn(Cell cell) {
|
||||
while (this.column != null) {
|
||||
int compare = Bytes.compareTo(column.getBuffer(), column.getOffset(), column.getLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
|
||||
int compare = CellComparator.compareQualifiers(cell, column.getBuffer(), column.getOffset(),
|
||||
column.getLength());
|
||||
resetTS();
|
||||
if (compare <= 0) {
|
||||
if (compare >= 0) {
|
||||
++this.index;
|
||||
if (done()) {
|
||||
// Will not hit any more columns in this storefile
|
||||
|
@ -232,8 +231,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
|
|||
} else {
|
||||
this.column = this.columns[this.index];
|
||||
}
|
||||
if (compare <= -1)
|
||||
continue;
|
||||
if (compare > 0) continue;
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -142,9 +142,7 @@ class GetClosestRowBeforeTracker {
|
|||
continue;
|
||||
}
|
||||
// Check column
|
||||
int ret = Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(),
|
||||
d.getQualifierArray(), d.getQualifierOffset(), d.getQualifierLength());
|
||||
int ret = CellComparator.compareQualifiers(kv, d);
|
||||
if (ret <= -1) {
|
||||
// This delete is for an earlier column.
|
||||
continue;
|
||||
|
|
|
@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
|
|||
import org.apache.hadoop.hbase.RegionTooBusyException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagRewriteCell;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
|
@ -3520,13 +3521,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
// Rewrite the cell with the updated set of tags
|
||||
|
||||
cells.set(i, new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
|
||||
cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
|
||||
cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
|
||||
newTags));
|
||||
cells.set(i, new TagRewriteCell(cell, Tag.fromList(newTags)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -6845,6 +6840,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
newCell.getQualifierArray(), newCell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
// copy in the value
|
||||
// TODO handle when oldCell is BBBacked
|
||||
System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
|
||||
newCell.getValueArray(), newCell.getValueOffset(),
|
||||
oldCell.getValueLength());
|
||||
|
@ -6866,15 +6862,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
List<Tag> newTags = new ArrayList<Tag>(1);
|
||||
newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(append.getTTL())));
|
||||
// Add the new TTL tag
|
||||
newCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength(),
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(),
|
||||
cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
|
||||
cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
|
||||
newTags);
|
||||
newCell = new TagRewriteCell(cell, Tag.fromList(newTags));
|
||||
} else {
|
||||
newCell = cell;
|
||||
}
|
||||
|
@ -7076,7 +7064,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
c = results.get(idx);
|
||||
ts = Math.max(now, c.getTimestamp());
|
||||
if(c.getValueLength() == Bytes.SIZEOF_LONG) {
|
||||
amount += Bytes.toLong(c.getValueArray(), c.getValueOffset(), Bytes.SIZEOF_LONG);
|
||||
amount += CellUtil.getValueAsLong(c);
|
||||
} else {
|
||||
// throw DoNotRetryIOException instead of IllegalArgumentException
|
||||
throw new org.apache.hadoop.hbase.DoNotRetryIOException(
|
||||
|
|
|
@ -1748,7 +1748,6 @@ public class HStore implements Store {
|
|||
*/
|
||||
static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now) {
|
||||
// Do not create an Iterator or Tag objects unless the cell actually has tags.
|
||||
// TODO: This check for tags is really expensive. We decode an int for key and value. Costs.
|
||||
if (cell.getTagsLength() > 0) {
|
||||
// Look for a TTL tag first. Use it instead of the family setting if
|
||||
// found. If a cell has multiple TTLs, resolve the conflict by using the
|
||||
|
|
|
@ -24,6 +24,8 @@ import java.util.TreeSet;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -72,8 +74,6 @@ public class ScanDeleteTracker implements DeleteTracker {
|
|||
@Override
|
||||
public void add(Cell cell) {
|
||||
long timestamp = cell.getTimestamp();
|
||||
int qualifierOffset = cell.getQualifierOffset();
|
||||
int qualifierLength = cell.getQualifierLength();
|
||||
byte type = cell.getTypeByte();
|
||||
if (!hasFamilyStamp || timestamp > familyStamp) {
|
||||
if (type == KeyValue.Type.DeleteFamily.getCode()) {
|
||||
|
@ -87,15 +87,14 @@ public class ScanDeleteTracker implements DeleteTracker {
|
|||
|
||||
if (deleteBuffer != null && type < deleteType) {
|
||||
// same column, so ignore less specific delete
|
||||
if (Bytes.equals(deleteBuffer, deleteOffset, deleteLength,
|
||||
cell.getQualifierArray(), qualifierOffset, qualifierLength)){
|
||||
if (CellUtil.matchingQualifier(cell, deleteBuffer, deleteOffset, deleteLength)) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
// new column, or more general delete type
|
||||
deleteBuffer = cell.getQualifierArray();
|
||||
deleteOffset = qualifierOffset;
|
||||
deleteLength = qualifierLength;
|
||||
deleteOffset = cell.getQualifierOffset();
|
||||
deleteLength = cell.getQualifierLength();
|
||||
deleteType = type;
|
||||
deleteTimestamp = timestamp;
|
||||
}
|
||||
|
@ -112,8 +111,6 @@ public class ScanDeleteTracker implements DeleteTracker {
|
|||
@Override
|
||||
public DeleteResult isDeleted(Cell cell) {
|
||||
long timestamp = cell.getTimestamp();
|
||||
int qualifierOffset = cell.getQualifierOffset();
|
||||
int qualifierLength = cell.getQualifierLength();
|
||||
if (hasFamilyStamp && timestamp <= familyStamp) {
|
||||
return DeleteResult.FAMILY_DELETED;
|
||||
}
|
||||
|
@ -123,9 +120,7 @@ public class ScanDeleteTracker implements DeleteTracker {
|
|||
}
|
||||
|
||||
if (deleteBuffer != null) {
|
||||
int ret = Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength,
|
||||
cell.getQualifierArray(), qualifierOffset, qualifierLength);
|
||||
|
||||
int ret = -(CellComparator.compareQualifiers(cell, deleteBuffer, deleteOffset, deleteLength));
|
||||
if (ret == 0) {
|
||||
if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {
|
||||
return DeleteResult.COLUMN_DELETED;
|
||||
|
@ -147,7 +142,8 @@ public class ScanDeleteTracker implements DeleteTracker {
|
|||
throw new IllegalStateException("isDelete failed: deleteBuffer="
|
||||
+ Bytes.toStringBinary(deleteBuffer, deleteOffset, deleteLength)
|
||||
+ ", qualifier="
|
||||
+ Bytes.toStringBinary(cell.getQualifierArray(), qualifierOffset, qualifierLength)
|
||||
+ Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength())
|
||||
+ ", timestamp=" + timestamp + ", comparison result: " + ret);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -544,23 +544,15 @@ public class ScanQueryMatcher {
|
|||
public Cell getKeyForNextColumn(Cell kv) {
|
||||
ColumnCount nextColumn = columns.getColumnHint();
|
||||
if (nextColumn == null) {
|
||||
return KeyValueUtil.createLastOnRow(
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
return CellUtil.createLastOnRowCol(kv);
|
||||
} else {
|
||||
return KeyValueUtil.createFirstOnRow(
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength());
|
||||
return CellUtil.createFirstOnRowCol(kv, nextColumn.getBuffer(), nextColumn.getOffset(),
|
||||
nextColumn.getLength());
|
||||
}
|
||||
}
|
||||
|
||||
public Cell getKeyForNextRow(Cell kv) {
|
||||
return KeyValueUtil.createLastOnRow(
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
null, 0, 0,
|
||||
null, 0, 0);
|
||||
public Cell getKeyForNextRow(Cell c) {
|
||||
return CellUtil.createLastOnRow(c);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -342,7 +342,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
// a higher timestamp than the max timestamp in this file. We know that
|
||||
// the next point when we have to consider this file again is when we
|
||||
// pass the max timestamp of this file (with the same row/column).
|
||||
setCurrentCell(KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
|
||||
setCurrentCell(CellUtil.createFirstOnRowColTS(kv, maxTimestampInFile));
|
||||
} else {
|
||||
// This will be the case e.g. when we need to seek to the next
|
||||
// row/column, and we don't know exactly what they are, so we set the
|
||||
|
@ -360,7 +360,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
// key/value and the store scanner will progress to the next column. This
|
||||
// is obviously not a "real real" seek, but unlike the fake KV earlier in
|
||||
// this method, we want this to be propagated to ScanQueryMatcher.
|
||||
setCurrentCell(KeyValueUtil.createLastOnRowCol(kv));
|
||||
setCurrentCell(CellUtil.createLastOnRowCol(kv));
|
||||
|
||||
realSeekDone = true;
|
||||
return true;
|
||||
|
@ -419,8 +419,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
public boolean seekToPreviousRow(Cell key) throws IOException {
|
||||
try {
|
||||
try {
|
||||
KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
|
||||
key.getRowLength());
|
||||
Cell seekKey = CellUtil.createFirstOnRow(key);
|
||||
if (seekCount != null) seekCount.incrementAndGet();
|
||||
if (!hfs.seekBefore(seekKey)) {
|
||||
this.cur = null;
|
||||
|
|
|
@ -797,8 +797,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
+ " followed by a " + "smaller key " + kv + " in cf " + store;
|
||||
}
|
||||
|
||||
protected boolean seekToNextRow(Cell kv) throws IOException {
|
||||
return reseek(KeyValueUtil.createLastOnRow(kv));
|
||||
protected boolean seekToNextRow(Cell c) throws IOException {
|
||||
return reseek(CellUtil.createLastOnRow(c));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.ByteRange;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
|
||||
|
||||
/**
|
||||
|
@ -98,8 +97,8 @@ class AccessControlFilter extends FilterBase {
|
|||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
if (prevFam.getBytes() == null
|
||||
|| (Bytes.compareTo(prevFam.getBytes(), prevFam.getOffset(), prevFam.getLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()) != 0)) {
|
||||
|| !(CellUtil.matchingFamily(cell, prevFam.getBytes(), prevFam.getOffset(),
|
||||
prevFam.getLength()))) {
|
||||
prevFam.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
|
||||
// Similar to VisibilityLabelFilter
|
||||
familyMaxVersions = cfVsMaxVersions.get(prevFam);
|
||||
|
@ -107,9 +106,8 @@ class AccessControlFilter extends FilterBase {
|
|||
prevQual.unset();
|
||||
}
|
||||
if (prevQual.getBytes() == null
|
||||
|| (Bytes.compareTo(prevQual.getBytes(), prevQual.getOffset(),
|
||||
prevQual.getLength(), cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength()) != 0)) {
|
||||
|| !(CellUtil.matchingQualifier(cell, prevQual.getBytes(), prevQual.getOffset(),
|
||||
prevQual.getLength()))) {
|
||||
prevQual.set(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
currentVersions = 0;
|
||||
|
|
|
@ -171,11 +171,10 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
|||
Map<String, List<Integer>> userAuths = new HashMap<String, List<Integer>>();
|
||||
for (List<Cell> cells : labelDetails) {
|
||||
for (Cell cell : cells) {
|
||||
if (Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), LABEL_QUALIFIER, 0, LABEL_QUALIFIER.length)) {
|
||||
if (CellUtil.matchingQualifier(cell, LABEL_QUALIFIER)) {
|
||||
labels.put(
|
||||
Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()),
|
||||
Bytes.toInt(cell.getRowArray(), cell.getRowOffset()));
|
||||
CellUtil.getRowAsInt(cell));
|
||||
} else {
|
||||
// These are user cells who has authorization for this label
|
||||
String user = Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
|
@ -185,7 +184,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
|||
auths = new ArrayList<Integer>();
|
||||
userAuths.put(user, auths);
|
||||
}
|
||||
auths.add(Bytes.toInt(cell.getRowArray(), cell.getRowOffset()));
|
||||
auths.add(CellUtil.getRowAsInt(cell));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -276,7 +275,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
|||
int labelOrdinal = this.labelsCache.getLabelOrdinal(authLabelStr);
|
||||
assert labelOrdinal > 0;
|
||||
Delete d = new Delete(Bytes.toBytes(labelOrdinal));
|
||||
d.deleteColumns(LABELS_TABLE_FAMILY, user);
|
||||
d.addColumns(LABELS_TABLE_FAMILY, user);
|
||||
deletes.add(d);
|
||||
} else {
|
||||
// This label is not set for the user.
|
||||
|
@ -341,7 +340,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
|||
scanner.next(results);
|
||||
if (results.isEmpty()) break;
|
||||
Cell cell = results.get(0);
|
||||
int ordinal = Bytes.toInt(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
int ordinal = CellUtil.getRowAsInt(cell);
|
||||
String label = this.labelsCache.getLabel(ordinal);
|
||||
if (label != null) {
|
||||
auths.add(label);
|
||||
|
@ -378,7 +377,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
|
|||
scanner.next(results);
|
||||
if (results.isEmpty()) break;
|
||||
Cell cell = results.get(0);
|
||||
int ordinal = Bytes.toInt(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
int ordinal = CellUtil.getRowAsInt(cell);
|
||||
String label = this.labelsCache.getLabel(ordinal);
|
||||
if (label != null) {
|
||||
auths.add(label);
|
||||
|
|
|
@ -22,9 +22,9 @@ import java.util.Map;
|
|||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.util.ByteRange;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
|
||||
|
||||
/**
|
||||
|
@ -58,8 +58,8 @@ class VisibilityLabelFilter extends FilterBase {
|
|||
@Override
|
||||
public ReturnCode filterKeyValue(Cell cell) throws IOException {
|
||||
if (curFamily.getBytes() == null
|
||||
|| (Bytes.compareTo(curFamily.getBytes(), curFamily.getOffset(), curFamily.getLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()) != 0)) {
|
||||
|| !(CellUtil.matchingFamily(cell, curFamily.getBytes(), curFamily.getOffset(),
|
||||
curFamily.getLength()))) {
|
||||
curFamily.set(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength());
|
||||
// For this family, all the columns can have max of curFamilyMaxVersions versions. No need to
|
||||
// consider the older versions for visibility label check.
|
||||
|
@ -69,9 +69,8 @@ class VisibilityLabelFilter extends FilterBase {
|
|||
curQualifier.unset();
|
||||
}
|
||||
if (curQualifier.getBytes() == null
|
||||
|| (Bytes.compareTo(curQualifier.getBytes(), curQualifier.getOffset(),
|
||||
curQualifier.getLength(), cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength()) != 0)) {
|
||||
|| !(CellUtil.matchingQualifier(cell, curQualifier.getBytes(), curQualifier.getOffset(),
|
||||
curQualifier.getLength()))) {
|
||||
curQualifier.set(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
curQualMetVersions = 0;
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
|
@ -71,8 +73,6 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
public void add(Cell delCell) {
|
||||
//Cannot call super.add because need to find if the delete needs to be considered
|
||||
long timestamp = delCell.getTimestamp();
|
||||
int qualifierOffset = delCell.getQualifierOffset();
|
||||
int qualifierLength = delCell.getQualifierLength();
|
||||
byte type = delCell.getTypeByte();
|
||||
if (type == KeyValue.Type.DeleteFamily.getCode()) {
|
||||
hasFamilyStamp = true;
|
||||
|
@ -88,8 +88,7 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
}
|
||||
// new column, or more general delete type
|
||||
if (deleteBuffer != null) {
|
||||
if (Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength, delCell.getQualifierArray(),
|
||||
qualifierOffset, qualifierLength) != 0) {
|
||||
if (!(CellUtil.matchingQualifier(delCell, deleteBuffer, deleteOffset, deleteLength))) {
|
||||
// A case where there are deletes for a column qualifier but there are
|
||||
// no corresponding puts for them. Rare case.
|
||||
visibilityTagsDeleteColumns = null;
|
||||
|
@ -106,8 +105,8 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
}
|
||||
}
|
||||
deleteBuffer = delCell.getQualifierArray();
|
||||
deleteOffset = qualifierOffset;
|
||||
deleteLength = qualifierLength;
|
||||
deleteOffset = delCell.getQualifierOffset();
|
||||
deleteLength = delCell.getQualifierLength();
|
||||
deleteType = type;
|
||||
deleteTimestamp = timestamp;
|
||||
extractDeleteCellVisTags(delCell, KeyValue.Type.codeToType(type));
|
||||
|
@ -189,8 +188,6 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
@Override
|
||||
public DeleteResult isDeleted(Cell cell) {
|
||||
long timestamp = cell.getTimestamp();
|
||||
int qualifierOffset = cell.getQualifierOffset();
|
||||
int qualifierLength = cell.getQualifierLength();
|
||||
try {
|
||||
if (hasFamilyStamp) {
|
||||
if (visibilityTagsDeleteFamily != null) {
|
||||
|
@ -247,9 +244,7 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
}
|
||||
}
|
||||
if (deleteBuffer != null) {
|
||||
int ret = Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength,
|
||||
cell.getQualifierArray(), qualifierOffset, qualifierLength);
|
||||
|
||||
int ret = CellComparator.compareQualifiers(cell, deleteBuffer, deleteOffset, deleteLength);
|
||||
if (ret == 0) {
|
||||
if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {
|
||||
if (visibilityTagsDeleteColumns != null) {
|
||||
|
@ -295,7 +290,7 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
}
|
||||
}
|
||||
}
|
||||
} else if (ret < 0) {
|
||||
} else if (ret > 0) {
|
||||
// Next column case.
|
||||
deleteBuffer = null;
|
||||
visibilityTagsDeleteColumns = null;
|
||||
|
@ -303,7 +298,8 @@ public class VisibilityScanDeleteTracker extends ScanDeleteTracker {
|
|||
} else {
|
||||
throw new IllegalStateException("isDeleted failed: deleteBuffer="
|
||||
+ Bytes.toStringBinary(deleteBuffer, deleteOffset, deleteLength) + ", qualifier="
|
||||
+ Bytes.toStringBinary(cell.getQualifierArray(), qualifierOffset, qualifierLength)
|
||||
+ Bytes.toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength())
|
||||
+ ", timestamp=" + timestamp + ", comparison result: " + ret);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -215,7 +215,7 @@ public class TestDataBlockEncoders {
|
|||
for (boolean seekBefore : new boolean[] { false, true }) {
|
||||
checkSeekingConsistency(encodedSeekers, seekBefore, sampleKv.get(sampleKv.size() - 1));
|
||||
KeyValue midKv = sampleKv.get(sampleKv.size() / 2);
|
||||
KeyValue lastMidKv =KeyValueUtil.createLastOnRowCol(midKv);
|
||||
Cell lastMidKv =CellUtil.createLastOnRowCol(midKv);
|
||||
checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv);
|
||||
}
|
||||
}
|
||||
|
@ -329,7 +329,7 @@ public class TestDataBlockEncoders {
|
|||
}
|
||||
|
||||
private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
|
||||
boolean seekBefore, KeyValue keyValue) {
|
||||
boolean seekBefore, Cell keyValue) {
|
||||
ByteBuffer expectedKeyValue = null;
|
||||
ByteBuffer expectedKey = null;
|
||||
ByteBuffer expectedValue = null;
|
||||
|
|
|
@ -25,9 +25,9 @@ import java.util.List;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
|
@ -136,8 +136,7 @@ public class TestSeekToBlockWithEncoders {
|
|||
KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"),
|
||||
Bytes.toBytes("q1"), Bytes.toBytes("val"));
|
||||
sampleKv.add(kv4);
|
||||
KeyValue toSeek = KeyValueUtil.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
|
||||
kv3.getRowLength(), null, 0, 0, null, 0, 0);
|
||||
Cell toSeek = CellUtil.createLastOnRow(kv3);
|
||||
seekToTheKey(kv3, sampleKv, toSeek);
|
||||
}
|
||||
|
||||
|
@ -250,7 +249,7 @@ public class TestSeekToBlockWithEncoders {
|
|||
seekToTheKey(kv5, sampleKv, toSeek);
|
||||
}
|
||||
|
||||
private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, KeyValue toSeek)
|
||||
private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, Cell toSeek)
|
||||
throws IOException {
|
||||
// create all seekers
|
||||
List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<DataBlockEncoder.EncodedSeeker>();
|
||||
|
@ -278,11 +277,9 @@ public class TestSeekToBlockWithEncoders {
|
|||
}
|
||||
|
||||
private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
|
||||
KeyValue keyValue, KeyValue expected) {
|
||||
Cell keyValue, KeyValue expected) {
|
||||
for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
|
||||
seeker.seekToKeyInBlock(
|
||||
new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue
|
||||
.getKeyLength()), false);
|
||||
seeker.seekToKeyInBlock(keyValue, false);
|
||||
Cell keyValue2 = seeker.getKeyValue();
|
||||
assertEquals(expected, keyValue2);
|
||||
seeker.rewind();
|
||||
|
|
|
@ -372,4 +372,32 @@ public class TestByteBufferUtils {
|
|||
assertEquals(i, Bytes.toInt(b, 3));
|
||||
assertEquals(l, Bytes.toLong(b, 7));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompareTo() {
|
||||
ByteBuffer bb1 = ByteBuffer.allocate(135);
|
||||
ByteBuffer bb2 = ByteBuffer.allocate(135);
|
||||
byte[] b = new byte[71];
|
||||
fillBB(bb1, (byte) 5);
|
||||
fillBB(bb2, (byte) 5);
|
||||
fillArray(b, (byte) 5);
|
||||
assertEquals(0, ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()));
|
||||
assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), b, 0, b.length) > 0);
|
||||
bb2.put(134, (byte) 6);
|
||||
assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) < 0);
|
||||
bb2.put(6, (byte) 4);
|
||||
assertTrue(ByteBufferUtils.compareTo(bb1, 0, bb1.remaining(), bb2, 0, bb2.remaining()) > 0);
|
||||
}
|
||||
|
||||
private static void fillBB(ByteBuffer bb, byte b) {
|
||||
for (int i = bb.position(); i < bb.limit(); i++) {
|
||||
bb.put(i, b);
|
||||
}
|
||||
}
|
||||
|
||||
private static void fillArray(byte[] bb, byte b) {
|
||||
for (int i = 0; i < bb.length; i++) {
|
||||
bb[i] = b;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue