HBASE-7319-Extend Cell usage through read path (Ram)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1585945 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
ramkrishna 2014-04-09 11:46:14 +00:00
parent 1b3d0c2656
commit 2922c5284d
68 changed files with 783 additions and 725 deletions

View File

@ -261,9 +261,8 @@ public class Result implements CellScannable {
}
for (int i = pos ; i < kvs.length ; i++ ) {
KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[i]);
if (kv.matchingColumn(family,qualifier)) {
result.add(kv);
if (CellUtil.matchingColumn(kvs[i], family,qualifier)) {
result.add(kvs[i]);
} else {
break;
}
@ -276,7 +275,7 @@ public class Result implements CellScannable {
final byte [] family,
final byte [] qualifier) {
Cell searchTerm =
KeyValue.createFirstOnRow(CellUtil.cloneRow(kvs[0]),
KeyValueUtil.createFirstOnRow(CellUtil.cloneRow(kvs[0]),
family, qualifier);
// pos === ( -(insertion point) - 1)
@ -317,7 +316,7 @@ public class Result implements CellScannable {
buffer = new byte[(int) Math.ceil(keyValueSize / PAD_WIDTH) * PAD_WIDTH];
}
Cell searchTerm = KeyValue.createFirstOnRow(buffer, 0,
Cell searchTerm = KeyValueUtil.createFirstOnRow(buffer, 0,
kvs[0].getRowArray(), kvs[0].getRowOffset(), kvs[0].getRowLength(),
family, foffset, flength,
qualifier, qoffset, qlength);
@ -361,9 +360,8 @@ public class Result implements CellScannable {
if (pos == -1) {
return null;
}
KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[pos]);
if (kv.matchingColumn(family, qualifier)) {
return kv;
if (CellUtil.matchingColumn(kvs[pos], family, qualifier)) {
return kvs[pos];
}
return null;
}
@ -402,9 +400,8 @@ public class Result implements CellScannable {
if (pos == -1) {
return null;
}
KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[pos]);
if (kv.matchingColumn(family, foffset, flength, qualifier, qoffset, qlength)) {
return kv;
if (CellUtil.matchingColumn(kvs[pos], family, foffset, flength, qualifier, qoffset, qlength)) {
return kvs[pos];
}
return null;
}

View File

@ -20,16 +20,16 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -145,7 +145,7 @@ public class ColumnPaginationFilter extends FilterBase
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
return KeyValueUtil.createFirstOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), columnOffset, 0, columnOffset.length);
}

View File

@ -21,16 +21,16 @@ package org.apache.hadoop.hbase.filter;
import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -131,7 +131,7 @@ public class ColumnPrefixFilter extends FilterBase {
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
return KeyValueUtil.createFirstOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), prefix, 0, prefix.length);
}

View File

@ -21,19 +21,19 @@ package org.apache.hadoop.hbase.filter;
import static org.apache.hadoop.hbase.util.Bytes.len;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used for selecting only those keys with columns that are
@ -217,7 +217,7 @@ public class ColumnRangeFilter extends FilterBase {
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(kv.getRowArray(), kv.getRowOffset(), kv
return KeyValueUtil.createFirstOnRow(kv.getRowArray(), kv.getRowOffset(), kv
.getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv
.getFamilyLength(), this.minColumn, 0, len(this.minColumn));

View File

@ -25,10 +25,10 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -139,7 +140,7 @@ public class DependentColumnFilter extends CompareFilter {
// TODO make matching Column a cell method or CellUtil method.
KeyValue v = KeyValueUtil.ensureKeyValue(c);
// Check if the column and qualifier match
if (!v.matchingColumn(this.columnFamily, this.columnQualifier)) {
if (!CellUtil.matchingColumn(v, this.columnFamily, this.columnQualifier)) {
// include non-matches for the time being, they'll be discarded afterwards
return ReturnCode.INCLUDE;
}

View File

@ -18,21 +18,20 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.Set;
import java.util.TreeSet;
import com.google.protobuf.ByteString;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* The filter looks for the given columns in KeyValue. Once there is a match for
@ -73,9 +72,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
private boolean hasOneMatchingQualifier(Cell v) {
for (byte[] q : qualifiers) {
// TODO get rid of this by adding matching qualifier to interface.
KeyValue kv = KeyValueUtil.ensureKeyValue(v);
if (kv.matchingQualifier(q)) {
if (CellUtil.matchingQualifier(v, q)) {
return true;
}
}

View File

@ -17,8 +17,9 @@
*/
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -31,9 +32,8 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Filters data based on fuzzy row key. Performs fast-forwards during scanning.
@ -131,7 +131,7 @@ public class FuzzyRowFilter extends FilterBase {
" currentKV: " + currentKV.toString());
}
return KeyValue.createFirstOnRow(nextRowKey);
return KeyValueUtil.createFirstOnRow(nextRowKey);
}
@Override

View File

@ -18,22 +18,22 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used for selecting only those keys with columns that matches
* a particular prefix. For example, if prefix is 'an', it will pass keys will
@ -157,7 +157,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
return KeyValueUtil.createFirstOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
}

View File

@ -19,10 +19,15 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -30,10 +35,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A {@link Filter} that checks a single column value, but does not emit the
@ -109,7 +111,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
KeyValue kv = KeyValueUtil.ensureKeyValue(it.next());
// If the current column is actually the tested column,
// we will skip it instead.
if (kv.matchingColumn(this.columnFamily, this.columnQualifier)) {
if (CellUtil.matchingColumn(kv, this.columnFamily, this.columnQualifier)) {
it.remove();
}
}

View File

@ -22,12 +22,12 @@ package org.apache.hadoop.hbase.filter;
import java.io.IOException;
import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
import com.google.protobuf.HBaseZeroCopyByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
@ -182,7 +183,7 @@ public class SingleColumnValueFilter extends FilterBase {
// We found but did not match the single column, skip to next row
return ReturnCode.NEXT_ROW;
}
if (!keyValue.matchingColumn(this.columnFamily, this.columnQualifier)) {
if (!CellUtil.matchingColumn(keyValue, this.columnFamily, this.columnQualifier)) {
return ReturnCode.INCLUDE;
}
foundColumn = true;

View File

@ -304,13 +304,19 @@ public final class CellUtil {
* @return True if the rows in <code>left</code> and <code>right</code> Cells match
*/
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 Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
right.getRowArray(), right.getRowOffset(), right.getRowLength());
}
public static boolean matchingRow(final Cell left, final byte[] buf) {
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
buf, 0, buf.length);
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, 0,
buf.length);
}
public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
final int length) {
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
length);
}
public static boolean matchingFamily(final Cell left, final Cell right) {
@ -319,20 +325,57 @@ public final class CellUtil {
}
public static boolean matchingFamily(final Cell left, final byte[] buf) {
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
buf, 0, buf.length);
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
0, buf.length);
}
public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
final int length) {
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
offset, length);
}
public static boolean matchingQualifier(final Cell left, final Cell right) {
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
right.getQualifierLength());
}
public static boolean matchingQualifier(final Cell left, final byte[] buf) {
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
buf, 0, buf.length);
if (buf == null) {
return left.getQualifierLength() == 0;
}
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
left.getQualifierLength(), buf, 0, buf.length);
}
public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
final int length) {
if (buf == null) {
return left.getQualifierLength() == 0;
}
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
left.getQualifierLength(), buf, offset, length);
}
public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
if (!matchingFamily(left, fam))
return false;
return matchingQualifier(left, qual);
}
public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
final int flength, final byte[] qual, final int qoffset, final int qlength) {
if (!matchingFamily(left, fam, foffset, flength))
return false;
return matchingQualifier(left, qual, qoffset, qlength);
}
public static boolean matchingColumn(final Cell left, final Cell right) {
if (!matchingFamily(left, right))
return false;
return matchingQualifier(left, right);
}
public static boolean matchingValue(final Cell left, final Cell right) {
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
@ -340,13 +383,14 @@ public final class CellUtil {
}
public static boolean matchingValue(final Cell left, final byte[] buf) {
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
buf, 0, buf.length);
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
buf.length);
}
/**
* @return True if a delete type, a {@link KeyValue.Type#Delete} or
* a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
* KeyValue type.
* @return True if a delete type, a {@link KeyValue.Type#Delete} or a
* {KeyValue.Type#DeleteFamily} or a
* {@link KeyValue.Type#DeleteColumn} KeyValue type.
*/
public static boolean isDelete(final Cell cell) {
return KeyValue.isDelete(cell.getTypeByte());
@ -356,6 +400,10 @@ public final class CellUtil {
return cell.getTypeByte() == Type.DeleteFamily.getCode();
}
public static boolean isDeleteFamilyVersion(final Cell cell) {
return cell.getTypeByte() == Type.DeleteFamilyVersion.getCode();
}
/**
* @param cell
* @return Estimate of the <code>cell</code> size in bytes.

View File

@ -863,7 +863,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* @throws IllegalArgumentException an illegal value was passed or there is insufficient space
* remaining in the buffer
*/
private static int writeByteArray(byte [] buffer, final int boffset,
public static int writeByteArray(byte [] buffer, final int boffset,
final byte [] row, final int roffset, final int rlength,
final byte [] family, final int foffset, int flength,
final byte [] qualifier, final int qoffset, int qlength,
@ -1618,94 +1618,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return bytes;
}
//---------------------------------------------------------------------------
//
// Compare specified fields against those contained in this KeyValue
//
//---------------------------------------------------------------------------
/**
* @param family
* @return True if matching families.
*/
public boolean matchingFamily(final byte [] family) {
if (this.length == 0 || this.bytes.length == 0) {
return false;
}
return Bytes.equals(family, 0, family.length,
this.bytes, getFamilyOffset(), getFamilyLength());
}
/**
* @param qualifier
* @return True if matching qualifiers.
*/
public boolean matchingQualifier(final byte [] qualifier) {
return matchingQualifier(qualifier, 0, qualifier.length);
}
public boolean matchingQualifier(final byte [] qualifier, int offset, int length) {
return Bytes.equals(qualifier, offset, length,
this.bytes, getQualifierOffset(), getQualifierLength());
}
public boolean matchingQualifier(final KeyValue other) {
return matchingQualifier(other.getQualifierArray(), other.getQualifierOffset(),
other.getQualifierLength());
}
public boolean matchingRow(final byte [] row) {
return matchingRow(row, 0, row.length);
}
public boolean matchingRow(final byte[] row, int offset, int length) {
return Bytes.equals(row, offset, length,
this.bytes, getRowOffset(), getRowLength());
}
public boolean matchingRow(KeyValue other) {
return matchingRow(other.getRowArray(), other.getRowOffset(),
other.getRowLength());
}
/**
*
* @param family column family
* @param qualifier column qualifier
* @return True if column matches
*/
public boolean matchingColumn(final byte[] family, final byte[] qualifier) {
return matchingColumn(family, 0, len(family), qualifier, 0, len(qualifier));
}
/**
* Checks if column matches.
*
* @param family family name
* @param foffset family offset
* @param flength family length
* @param qualifier column qualifier
* @param qoffset qualifier offset
* @param qlength qualifier length
*
* @return True if column matches
*/
public boolean matchingColumn(final byte [] family, final int foffset, final int flength,
final byte [] qualifier, final int qoffset, final int qlength) {
int rl = getRowLength();
int o = getFamilyOffset(rl);
int fl = getFamilyLength(o);
if (!Bytes.equals(family, foffset, flength, this.bytes, o, fl)) {
return false;
}
int ql = getQualifierLength(rl, fl);
if (qualifier == null || qlength == 0) {
return (ql == 0);
}
return Bytes.equals(qualifier, qoffset, qlength, this.bytes, o + fl, ql);
}
/**
* Creates a new KeyValue that only contains the key portion (the value is
* set to be null).
@ -2398,247 +2310,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
}
/**
* Creates a KeyValue that is last on the specified row id. That is,
* every other possible KeyValue for the given row would compareTo()
* less than the result of this call.
* @param row row key
* @return Last possible KeyValue on passed <code>row</code>
*/
public static KeyValue createLastOnRow(final byte[] row) {
return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
}
/**
* Create a KeyValue that is smaller than all other possible KeyValues
* for the given row. That is any (valid) KeyValue on 'row' would sort
* _after_ the result.
*
* @param row - row key (arbitrary byte array)
* @return First possible KeyValue on passed <code>row</code>
*/
public static KeyValue createFirstOnRow(final byte [] row) {
return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
}
/**
* Create a KeyValue that is smaller than all other possible KeyValues
* for the given row. That is any (valid) KeyValue on 'row' would sort
* _after_ the result.
*
* @param row - row key (arbitrary byte array)
* @return First possible KeyValue on passed <code>row</code>
*/
public static KeyValue createFirstOnRow(final byte [] row, int roffset, short rlength) {
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 smaller than all other KeyValues that
* are older than the passed timestamp.
* @param row - row key (arbitrary byte array)
* @param ts - timestamp
* @return First possible key on passed <code>row</code> and timestamp.
*/
public static KeyValue createFirstOnRow(final byte [] row,
final long ts) {
return new KeyValue(row, null, null, ts, Type.Maximum);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,family,qualifier.
* Used for seeking.
* @param row - row key (arbitrary byte array)
* @param family - family name
* @param qualifier - column qualifier
* @return First possible key on passed <code>row</code>, and column.
*/
public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
final byte [] qualifier) {
return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
}
/**
* Create a Delete Family KeyValue for the specified row and family that would
* be smaller than all other possible Delete Family KeyValues that have the
* same row and family.
* Used for seeking.
* @param row - row key (arbitrary byte array)
* @param family - family name
* @return First Delete Family possible key on passed <code>row</code>.
*/
public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
final byte [] family) {
return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
Type.DeleteFamily);
}
/**
* @param row - row key (arbitrary byte array)
* @param f - family name
* @param q - column qualifier
* @param ts - timestamp
* @return First possible key on passed <code>row</code>, column and timestamp
*/
public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
final byte [] q, final long ts) {
return new KeyValue(row, f, q, ts, Type.Maximum);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,
* family, qualifier.
* Used for seeking.
* @param row row key
* @param roffset row offset
* @param rlength row length
* @param family family name
* @param foffset family offset
* @param flength family length
* @param qualifier column qualifier
* @param qoffset qualifier offset
* @param qlength qualifier length
* @return First possible key on passed Row, Family, Qualifier.
*/
public static KeyValue createFirstOnRow(final byte [] row,
final int roffset, final int rlength, final byte [] family,
final int foffset, final int flength, final byte [] qualifier,
final int qoffset, final int qlength) {
return new KeyValue(row, roffset, rlength, family,
foffset, flength, qualifier, qoffset, qlength,
HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,
* family, qualifier.
* Used for seeking.
*
* @param buffer the buffer to use for the new <code>KeyValue</code> object
* @param row the value key
* @param family family name
* @param qualifier column qualifier
*
* @return First possible key on passed Row, Family, Qualifier.
*
* @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
* than the provided buffer or than <code>Integer.MAX_VALUE</code>
*/
public static KeyValue createFirstOnRow(byte [] buffer, final byte [] row,
final byte [] family, final byte [] qualifier)
throws IllegalArgumentException {
return createFirstOnRow(buffer, 0, row, 0, row.length,
family, 0, family.length,
qualifier, 0, qualifier.length);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,
* family, qualifier.
* Used for seeking.
*
* @param buffer the buffer to use for the new <code>KeyValue</code> object
* @param boffset buffer offset
* @param row the value key
* @param roffset row offset
* @param rlength row length
* @param family family name
* @param foffset family offset
* @param flength family length
* @param qualifier column qualifier
* @param qoffset qualifier offset
* @param qlength qualifier length
*
* @return First possible key on passed Row, Family, Qualifier.
*
* @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
* than the provided buffer or than <code>Integer.MAX_VALUE</code>
*/
public static KeyValue createFirstOnRow(byte [] buffer, final int boffset,
final byte [] row, final int roffset, final int rlength,
final byte [] family, final int foffset, final int flength,
final byte [] qualifier, final int qoffset, final int qlength)
throws IllegalArgumentException {
long lLength = getKeyValueDataStructureSize(rlength, flength, qlength, 0);
if (lLength > Integer.MAX_VALUE) {
throw new IllegalArgumentException("KeyValue length " + lLength + " > " + Integer.MAX_VALUE);
}
int iLength = (int) lLength;
if (buffer.length - boffset < iLength) {
throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < " +
iLength);
}
int len = writeByteArray(buffer, boffset, row, roffset, rlength, family, foffset, flength,
qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum,
null, 0, 0, null);
return new KeyValue(buffer, boffset, len);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* larger than or equal to all other possible KeyValues that have the same
* row, family, qualifier.
* Used for reseeking.
* @param row row key
* @param roffset row offset
* @param rlength row length
* @param family family name
* @param foffset family offset
* @param flength family length
* @param qualifier column qualifier
* @param qoffset qualifier offset
* @param qlength qualifier length
* @return Last possible key on passed row, family, qualifier.
*/
public static KeyValue createLastOnRow(final byte [] row,
final int roffset, final int rlength, final byte [] family,
final int foffset, final int flength, final byte [] qualifier,
final int qoffset, final int qlength) {
return new KeyValue(row, roffset, rlength, family,
foffset, flength, qualifier, qoffset, qlength,
HConstants.OLDEST_TIMESTAMP, Type.Minimum, 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.
* @return the last key on the row/column of the given key-value pair
*/
public KeyValue createLastOnRowCol() {
return new KeyValue(
bytes, getRowOffset(), getRowLength(),
bytes, getFamilyOffset(), getFamilyLength(),
bytes, getQualifierOffset(), 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.
*/
public KeyValue createFirstOnRowColTS(long ts) {
return new KeyValue(
bytes, getRowOffset(), getRowLength(),
bytes, getFamilyOffset(), getFamilyLength(),
bytes, getQualifierOffset(), getQualifierLength(),
ts, Type.Maximum, bytes, getValueOffset(), getValueLength());
}
/**
* @param b
* @return A KeyValue made of a byte array that holds the key-only part.

View File

@ -23,6 +23,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.IterableUtils;
@ -189,7 +190,7 @@ public class KeyValueUtil {
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 KeyValue.createFirstOnRow(nextRow);
return createFirstOnRow(nextRow);
}
/**
@ -199,7 +200,7 @@ public class KeyValueUtil {
byte[] thisRow = new SimpleByteRange(in.getRowArray(), in.getRowOffset(), in.getRowLength())
.deepCopyToNewArray();
byte[] nextRow = Bytes.unsignedCopyAndIncrement(thisRow);
return KeyValue.createFirstOnRow(nextRow);
return createFirstOnRow(nextRow);
}
/**
@ -210,9 +211,287 @@ public class KeyValueUtil {
* @return previous key
*/
public static KeyValue previousKey(final KeyValue in) {
return KeyValue.createFirstOnRow(CellUtil.cloneRow(in), CellUtil.cloneFamily(in),
return createFirstOnRow(CellUtil.cloneRow(in), CellUtil.cloneFamily(in),
CellUtil.cloneQualifier(in), in.getTimestamp() - 1);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* larger than or equal to all other possible KeyValues that have the same
* row, family, qualifier. Used for reseeking.
*
* @param row
* row key
* @param roffset
* row offset
* @param rlength
* row length
* @param family
* family name
* @param foffset
* family offset
* @param flength
* family length
* @param qualifier
* column qualifier
* @param qoffset
* qualifier offset
* @param qlength
* qualifier length
* @return Last possible key on passed row, family, qualifier.
*/
public static KeyValue createLastOnRow(final byte[] row, final int roffset, final int rlength,
final byte[] family, final int foffset, final int flength, final byte[] qualifier,
final int qoffset, final int qlength) {
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
* _after_ the result.
*
* @param row - row key (arbitrary byte array)
* @return First possible KeyValue on passed <code>row</code>
*/
public static KeyValue createFirstOnRow(final byte [] row, int roffset, short rlength) {
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,
* every other possible KeyValue for the given row would compareTo()
* less than the result of this call.
* @param row row key
* @return Last possible KeyValue on passed <code>row</code>
*/
public static KeyValue createLastOnRow(final byte[] row) {
return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
}
/**
* Create a KeyValue that is smaller than all other possible KeyValues
* for the given row. That is any (valid) KeyValue on 'row' would sort
* _after_ the result.
*
* @param row - row key (arbitrary byte array)
* @return First possible KeyValue on passed <code>row</code>
*/
public static KeyValue createFirstOnRow(final byte [] row) {
return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
}
/**
* Creates a KeyValue that is smaller than all other KeyValues that
* are older than the passed timestamp.
* @param row - row key (arbitrary byte array)
* @param ts - timestamp
* @return First possible key on passed <code>row</code> and timestamp.
*/
public static KeyValue createFirstOnRow(final byte [] row,
final long ts) {
return new KeyValue(row, null, null, ts, Type.Maximum);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,family,qualifier.
* Used for seeking.
* @param row - row key (arbitrary byte array)
* @param family - family name
* @param qualifier - column qualifier
* @return First possible key on passed <code>row</code>, and column.
*/
public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
final byte [] qualifier) {
return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
}
/**
* Create a Delete Family KeyValue for the specified row and family that would
* be smaller than all other possible Delete Family KeyValues that have the
* same row and family.
* Used for seeking.
* @param row - row key (arbitrary byte array)
* @param family - family name
* @return First Delete Family possible key on passed <code>row</code>.
*/
public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
final byte [] family) {
return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
Type.DeleteFamily);
}
/**
* @param row - row key (arbitrary byte array)
* @param f - family name
* @param q - column qualifier
* @param ts - timestamp
* @return First possible key on passed <code>row</code>, column and timestamp
*/
public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
final byte [] q, final long ts) {
return new KeyValue(row, f, q, ts, Type.Maximum);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,
* family, qualifier.
* Used for seeking.
* @param row row key
* @param roffset row offset
* @param rlength row length
* @param family family name
* @param foffset family offset
* @param flength family length
* @param qualifier column qualifier
* @param qoffset qualifier offset
* @param qlength qualifier length
* @return First possible key on passed Row, Family, Qualifier.
*/
public static KeyValue createFirstOnRow(final byte [] row,
final int roffset, final int rlength, final byte [] family,
final int foffset, final int flength, final byte [] qualifier,
final int qoffset, final int qlength) {
return new KeyValue(row, roffset, rlength, family,
foffset, flength, qualifier, qoffset, qlength,
HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,
* family, qualifier.
* Used for seeking.
*
* @param buffer the buffer to use for the new <code>KeyValue</code> object
* @param row the value key
* @param family family name
* @param qualifier column qualifier
*
* @return First possible key on passed Row, Family, Qualifier.
*
* @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
* than the provided buffer or than <code>Integer.MAX_VALUE</code>
*/
public static KeyValue createFirstOnRow(byte [] buffer, final byte [] row,
final byte [] family, final byte [] qualifier)
throws IllegalArgumentException {
return createFirstOnRow(buffer, 0, row, 0, row.length,
family, 0, family.length,
qualifier, 0, qualifier.length);
}
/**
* Create a KeyValue for the specified row, family and qualifier that would be
* smaller than all other possible KeyValues that have the same row,
* family, qualifier.
* Used for seeking.
*
* @param buffer the buffer to use for the new <code>KeyValue</code> object
* @param boffset buffer offset
* @param row the value key
* @param roffset row offset
* @param rlength row length
* @param family family name
* @param foffset family offset
* @param flength family length
* @param qualifier column qualifier
* @param qoffset qualifier offset
* @param qlength qualifier length
*
* @return First possible key on passed Row, Family, Qualifier.
*
* @throws IllegalArgumentException The resulting <code>KeyValue</code> object would be larger
* than the provided buffer or than <code>Integer.MAX_VALUE</code>
*/
public static KeyValue createFirstOnRow(byte[] buffer, final int boffset, final byte[] row,
final int roffset, final int rlength, final byte[] family, final int foffset,
final int flength, final byte[] qualifier, final int qoffset, final int qlength)
throws IllegalArgumentException {
long lLength = KeyValue.getKeyValueDataStructureSize(rlength, flength, qlength, 0);
if (lLength > Integer.MAX_VALUE) {
throw new IllegalArgumentException("KeyValue length " + lLength + " > " + Integer.MAX_VALUE);
}
int iLength = (int) lLength;
if (buffer.length - boffset < iLength) {
throw new IllegalArgumentException("Buffer size " + (buffer.length - boffset) + " < "
+ iLength);
}
int len = KeyValue.writeByteArray(buffer, boffset, row, roffset, rlength, family, foffset,
flength, qualifier, qoffset, qlength, HConstants.LATEST_TIMESTAMP, KeyValue.Type.Maximum,
null, 0, 0, null);
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 **********************************/
/**
@ -233,4 +512,5 @@ public class KeyValueUtil {
});
return new ArrayList<KeyValue>(lazyList);
}
}

View File

@ -259,7 +259,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
}
@Override
public KeyValue getKeyValue() {
public Cell getKeyValue() {
ByteBuffer kvBuf = getKeyValueBuffer();
KeyValue kv = new KeyValue(kvBuf.array(), kvBuf.arrayOffset(), kvBuf.array().length
- kvBuf.arrayOffset());

View File

@ -22,7 +22,6 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -147,7 +146,7 @@ public interface DataBlockEncoder {
* @return the KeyValue object at the current position. Includes memstore
* timestamp.
*/
KeyValue getKeyValue();
Cell getKeyValue();
/** Set position to beginning of given block */
void rewind();

View File

@ -45,15 +45,15 @@ public class TestKeyValue extends TestCase {
byte [] qualifier2 = Bytes.toBytes("ef");
KeyValue aaa = new KeyValue(a, family1, qualifier1, 0L, Type.Put, a);
assertFalse(aaa.matchingColumn(family2, qualifier2));
assertTrue(aaa.matchingColumn(family1, qualifier1));
assertFalse(CellUtil.matchingColumn(aaa, family2, qualifier2));
assertTrue(CellUtil.matchingColumn(aaa, family1, qualifier1));
aaa = new KeyValue(a, family2, qualifier2, 0L, Type.Put, a);
assertFalse(aaa.matchingColumn(family1, qualifier1));
assertTrue(aaa.matchingColumn(family2,qualifier2));
assertFalse(CellUtil.matchingColumn(aaa, family1, qualifier1));
assertTrue(CellUtil.matchingColumn(aaa, family2,qualifier2));
byte [] nullQualifier = new byte[0];
aaa = new KeyValue(a, family1, nullQualifier, 0L, Type.Put, a);
assertTrue(aaa.matchingColumn(family1,null));
assertFalse(aaa.matchingColumn(family2,qualifier2));
assertTrue(CellUtil.matchingColumn(aaa, family1,null));
assertFalse(CellUtil.matchingColumn(aaa, family2,qualifier2));
}
/**
@ -68,7 +68,7 @@ public class TestKeyValue extends TestCase {
byte [] qualifier2 = Bytes.toBytes("def");
KeyValue aaa = new KeyValue(a, family1, qualifier1, 0L, Type.Put, a);
assertFalse(aaa.matchingColumn(family2, qualifier2));
assertFalse(CellUtil.matchingColumn(aaa, family2, qualifier2));
}
public void testBasics() throws Exception {
@ -90,7 +90,7 @@ public class TestKeyValue extends TestCase {
final long timestamp, final byte [] value) {
KeyValue kv = new KeyValue(row, family, qualifier, timestamp, value);
assertTrue(Bytes.compareTo(kv.getRow(), row) == 0);
assertTrue(kv.matchingColumn(family, qualifier));
assertTrue(CellUtil.matchingColumn(kv, family, qualifier));
// Call toString to make sure it works.
LOG.info(kv.toString());
}
@ -363,15 +363,15 @@ public class TestKeyValue extends TestCase {
// These are listed in sort order (ie: every one should be less
// than the one on the next line).
final KeyValue firstOnRowA = KeyValue.createFirstOnRow(rowA);
final KeyValue firstOnRowABufferFamQual = KeyValue.createFirstOnRow(bufferA, offsetA,
final KeyValue firstOnRowA = KeyValueUtil.createFirstOnRow(rowA);
final KeyValue firstOnRowABufferFamQual = KeyValueUtil.createFirstOnRow(bufferA, offsetA,
rowA, 0, rowA.length, family, 0, family.length, qualA, 0, qualA.length);
final KeyValue kvA_1 = new KeyValue(rowA, null, null, ts, Type.Put);
final KeyValue kvA_2 = new KeyValue(rowA, family, qualA, ts, Type.Put);
final KeyValue lastOnRowA = KeyValue.createLastOnRow(rowA);
final KeyValue firstOnRowB = KeyValue.createFirstOnRow(rowB);
final KeyValue firstOnRowBBufferFam = KeyValue.createFirstOnRow(bufferB, offsetB,
final KeyValue lastOnRowA = KeyValueUtil.createLastOnRow(rowA);
final KeyValue firstOnRowB = KeyValueUtil.createFirstOnRow(rowB);
final KeyValue firstOnRowBBufferFam = KeyValueUtil.createFirstOnRow(bufferB, offsetB,
rowB, 0, rowB.length, family, 0, family.length, null, 0, 0);
final KeyValue kvB = new KeyValue(rowB, family, qualA, ts, Type.Put);

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.row.data;
import java.util.List;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
@ -72,7 +73,7 @@ public class TestRowDataDeeper extends BaseTestRowData{
* The searcher should get a token mismatch on the "r" branch. Assert that it skips not only
* rA, but rB as well.
*/
KeyValue cfcRow = KeyValue.createFirstOnRow(Bytes.toBytes("cfc"));
KeyValue cfcRow = KeyValueUtil.createFirstOnRow(Bytes.toBytes("cfc"));
CellScannerPosition position = searcher.positionAtOrAfter(cfcRow);
Assert.assertEquals(CellScannerPosition.AFTER, position);
Assert.assertEquals(d.get(2), searcher.current());

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.row.data;
import java.util.List;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
@ -65,7 +66,7 @@ public class TestRowDataTrivial extends BaseTestRowData{
* The searcher should get a token mismatch on the "r" branch. Assert that it skips not only rA,
* but rB as well.
*/
KeyValue afterLast = KeyValue.createFirstOnRow(Bytes.toBytes("zzz"));
KeyValue afterLast = KeyValueUtil.createFirstOnRow(Bytes.toBytes("zzz"));
CellScannerPosition position = searcher.positionAtOrAfter(afterLast);
Assert.assertEquals(CellScannerPosition.AFTER_LAST, position);
Assert.assertNull(searcher.current());

View File

@ -21,6 +21,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
@ -71,7 +72,7 @@ public class TestRowDataTrivialWithTags extends BaseTestRowData{
* The searcher should get a token mismatch on the "r" branch. Assert that
* it skips not only rA, but rB as well.
*/
KeyValue afterLast = KeyValue.createFirstOnRow(Bytes.toBytes("zzz"));
KeyValue afterLast = KeyValueUtil.createFirstOnRow(Bytes.toBytes("zzz"));
CellScannerPosition position = searcher.positionAtOrAfter(afterLast);
Assert.assertEquals(CellScannerPosition.AFTER_LAST, position);
Assert.assertNull(searcher.current());

View File

@ -148,7 +148,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
return delegate.getValueString();
}
public KeyValue getKeyValue() {
public Cell getKeyValue() {
if (atEnd) return null;
return delegate.getKeyValue();

View File

@ -24,16 +24,17 @@ import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.HBaseZeroCopyByteString;
/**
* A reference to the top or bottom half of a store file where 'bottom' is the first half
* of the file containing the keys that sort lowest and 'top' is the second half
@ -90,7 +91,7 @@ public class Reference {
* @param fr
*/
Reference(final byte [] splitRow, final Range fr) {
this.splitkey = splitRow == null? null: KeyValue.createFirstOnRow(splitRow).getKey();
this.splitkey = splitRow == null? null: KeyValueUtil.createFirstOnRow(splitRow).getKey();
this.region = fr;
}

View File

@ -29,9 +29,6 @@ import java.util.Locale;
import java.util.Map;
import java.util.SortedMap;
import com.yammer.metrics.core.*;
import com.yammer.metrics.reporting.ConsoleReporter;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
@ -45,10 +42,11 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
@ -59,6 +57,13 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Writables;
import com.yammer.metrics.core.Histogram;
import com.yammer.metrics.core.Metric;
import com.yammer.metrics.core.MetricName;
import com.yammer.metrics.core.MetricPredicate;
import com.yammer.metrics.core.MetricsRegistry;
import com.yammer.metrics.reporting.ConsoleReporter;
/**
* Implements pretty-printing functionality for {@link HFile}s.
*/
@ -228,7 +233,7 @@ public class HFilePrettyPrinter {
if (this.isSeekToRow) {
// seek to the first kv on this row
shouldScanKeysValues =
(scanner.seekTo(KeyValue.createFirstOnRow(this.row).getKey()) != -1);
(scanner.seekTo(KeyValueUtil.createFirstOnRow(this.row).getKey()) != -1);
} else {
shouldScanKeysValues = scanner.seekTo();
}
@ -258,7 +263,7 @@ public class HFilePrettyPrinter {
HFileScanner scanner, byte[] row) throws IOException {
KeyValue pkv = null;
do {
KeyValue kv = scanner.getKeyValue();
KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue());
if (row != null && row.length != 0) {
int result = Bytes.compareTo(kv.getRow(), row);
if (result > 0) {

View File

@ -716,7 +716,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
@Override
public KeyValue getKeyValue() {
public Cell getKeyValue() {
if (!isSeeked())
return null;
@ -1178,7 +1178,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
@Override
public KeyValue getKeyValue() {
public Cell getKeyValue() {
if (block == null) {
return null;
}

View File

@ -134,7 +134,7 @@ public interface HFileScanner {
/**
* @return Instance of {@link KeyValue}.
*/
KeyValue getKeyValue();
Cell getKeyValue();
/**
* Convenience method to get a copy of the key as a string - interpreting the
* bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.

View File

@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -734,7 +735,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
HFileScanner scanner = halfReader.getScanner(false, false, false);
scanner.seekTo();
do {
KeyValue kv = scanner.getKeyValue();
KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue());
halfWriter.append(kv);
} while (scanner.next());

View File

@ -28,9 +28,10 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
@ -133,7 +134,8 @@ public class WALPlayer extends Configured implements Tool {
// multiple rows (HBASE-5229).
// Aggregate as much as possible into a single Put/Delete
// operation before writing to the context.
if (lastKV == null || lastKV.getType() != kv.getType() || !lastKV.matchingRow(kv)) {
if (lastKV == null || lastKV.getType() != kv.getType()
|| !CellUtil.matchingRow(lastKV, kv)) {
// row or type changed, write out aggregate KVs.
if (put != null) context.write(tableOut, put);
if (del != null) context.write(tableOut, del);

View File

@ -35,6 +35,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
@ -463,14 +464,14 @@ public class DefaultMemStore implements MemStore {
byte[] qualifier,
long newValue,
long now) {
KeyValue firstKv = KeyValue.createFirstOnRow(
KeyValue firstKv = KeyValueUtil.createFirstOnRow(
row, family, qualifier);
// Is there a KeyValue in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
if (!snSs.isEmpty()) {
KeyValue snKv = snSs.first();
// is there a matching KV in the snapshot?
if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
if (CellUtil.matchingRow(snKv, firstKv) && CellUtil.matchingQualifier(snKv, firstKv)) {
if (snKv.getTimestamp() == now) {
// poop,
now += 1;
@ -487,13 +488,13 @@ public class DefaultMemStore implements MemStore {
SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
for (KeyValue kv : ss) {
// if this isnt the row we are interested in, then bail:
if (!kv.matchingColumn(family, qualifier) || !kv.matchingRow(firstKv)) {
if (!CellUtil.matchingColumn(kv, family, qualifier) || !CellUtil.matchingRow(kv, firstKv)) {
break; // rows dont match, bail.
}
// if the qualifier matches and it's a put, just RM it out of the kvset.
if (kv.getTypeByte() == KeyValue.Type.Put.getCode() &&
kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
kv.getTimestamp() > now && CellUtil.matchingQualifier(firstKv, kv)) {
now = kv.getTimestamp();
}
}
@ -558,7 +559,7 @@ public class DefaultMemStore implements MemStore {
// Get the KeyValues for the row/family/qualifier regardless of timestamp.
// For this case we want to clean up any other puts
KeyValue firstKv = KeyValue.createFirstOnRow(
KeyValue firstKv = KeyValueUtil.createFirstOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
@ -574,7 +575,7 @@ public class DefaultMemStore implements MemStore {
continue;
}
// check that this is the row and column we are interested in, otherwise bail
if (kv.matchingRow(cur) && kv.matchingQualifier(cur)) {
if (CellUtil.matchingRow(kv, cur) && CellUtil.matchingQualifier(kv, cur)) {
// only remove Puts that concurrent scanners cannot possibly see
if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
cur.getMvccVersion() <= readpoint) {
@ -766,20 +767,20 @@ public class DefaultMemStore implements MemStore {
* @return false if the key is null or if there is no data
*/
@Override
public synchronized boolean seek(KeyValue key) {
public synchronized boolean seek(Cell key) {
if (key == null) {
close();
return false;
}
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
// kvset and snapshot will never be null.
// if tailSet can't find anything, SortedSet is empty (not null).
kvsetIt = kvsetAtCreation.tailSet(key).iterator();
snapshotIt = snapshotAtCreation.tailSet(key).iterator();
kvsetIt = kvsetAtCreation.tailSet(kv).iterator();
snapshotIt = snapshotAtCreation.tailSet(kv).iterator();
kvsetItRow = null;
snapshotItRow = null;
return seekInSubLists(key);
return seekInSubLists(kv);
}
@ -804,7 +805,7 @@ public class DefaultMemStore implements MemStore {
* @return true if there is at least one KV to read, false otherwise
*/
@Override
public synchronized boolean reseek(KeyValue key) {
public synchronized boolean reseek(Cell key) {
/*
See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation.
This code is executed concurrently with flush and puts, without locks.
@ -817,11 +818,11 @@ public class DefaultMemStore implements MemStore {
get it. So we remember the last keys we iterated to and restore
the reseeked set to at least that point.
*/
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
kvsetIt = kvsetAtCreation.tailSet(getHighest(kv, kvsetItRow)).iterator();
snapshotIt = snapshotAtCreation.tailSet(getHighest(kv, snapshotItRow)).iterator();
kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
return seekInSubLists(key);
return seekInSubLists(kv);
}
@ -928,7 +929,7 @@ public class DefaultMemStore implements MemStore {
* the scanner to the previous row of given key
*/
@Override
public synchronized boolean backwardSeek(KeyValue key) {
public synchronized boolean backwardSeek(Cell key) {
seek(key);
if (peek() == null || comparator.compareRows(peek(), key) > 0) {
return seekToPreviousRow(key);
@ -942,8 +943,9 @@ public class DefaultMemStore implements MemStore {
* specified key, then seek to the first KeyValue of previous row
*/
@Override
public synchronized boolean seekToPreviousRow(KeyValue key) {
KeyValue firstKeyOnRow = KeyValue.createFirstOnRow(key.getRow());
public synchronized boolean seekToPreviousRow(Cell key) {
KeyValue firstKeyOnRow = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
key.getRowLength());
SortedSet<KeyValue> kvHead = kvsetAtCreation.headSet(firstKeyOnRow);
KeyValue kvsetBeforeRow = kvHead.isEmpty() ? null : kvHead.last();
SortedSet<KeyValue> snapshotHead = snapshotAtCreation
@ -955,8 +957,8 @@ public class DefaultMemStore implements MemStore {
theNext = null;
return false;
}
KeyValue firstKeyOnPreviousRow = KeyValue
.createFirstOnRow(lastKVBeforeRow.getRow());
KeyValue firstKeyOnPreviousRow = KeyValueUtil.createFirstOnRow(lastKVBeforeRow.getRowArray(),
lastKVBeforeRow.getRowOffset(), lastKVBeforeRow.getRowLength());
this.stopSkippingKVsIfNextRow = true;
seek(firstKeyOnPreviousRow);
this.stopSkippingKVsIfNextRow = false;
@ -977,7 +979,8 @@ public class DefaultMemStore implements MemStore {
if (higherKv == null) {
return false;
}
KeyValue firstKvOnLastRow = KeyValue.createFirstOnRow(higherKv.getRow());
KeyValue firstKvOnLastRow = KeyValueUtil.createFirstOnRow(higherKv.getRowArray(),
higherKv.getRowOffset(), higherKv.getRowLength());
if (seek(firstKvOnLastRow)) {
return true;
} else {

View File

@ -24,9 +24,11 @@ import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -40,7 +42,7 @@ class GetClosestRowBeforeTracker {
private final KeyValue targetkey;
// Any cell w/ a ts older than this is expired.
private final long oldestts;
private KeyValue candidate = null;
private Cell candidate = null;
private final KVComparator kvcomparator;
// Flag for whether we're doing getclosest on a metaregion.
private final boolean metaregion;
@ -82,7 +84,7 @@ class GetClosestRowBeforeTracker {
* @param kv
* @return True if this <code>kv</code> is expired.
*/
boolean isExpired(final KeyValue kv) {
boolean isExpired(final Cell kv) {
return HStore.isExpired(kv, this.oldestts);
}
@ -90,20 +92,20 @@ class GetClosestRowBeforeTracker {
* Add the specified KeyValue to the list of deletes.
* @param kv
*/
private void addDelete(final KeyValue kv) {
private void addDelete(final Cell kv) {
NavigableSet<KeyValue> rowdeletes = this.deletes.get(kv);
if (rowdeletes == null) {
rowdeletes = new TreeSet<KeyValue>(this.kvcomparator);
this.deletes.put(kv, rowdeletes);
this.deletes.put(KeyValueUtil.ensureKeyValue(kv), rowdeletes);
}
rowdeletes.add(kv);
rowdeletes.add(KeyValueUtil.ensureKeyValue(kv));
}
/*
* @param kv Adds candidate if nearer the target than previous candidate.
* @return True if updated candidate.
*/
private boolean addCandidate(final KeyValue kv) {
private boolean addCandidate(final Cell kv) {
if (!isDeleted(kv) && isBetterCandidate(kv)) {
this.candidate = kv;
return true;
@ -111,7 +113,7 @@ class GetClosestRowBeforeTracker {
return false;
}
boolean isBetterCandidate(final KeyValue contender) {
boolean isBetterCandidate(final Cell contender) {
return this.candidate == null ||
(this.kvcomparator.compareRows(this.candidate, contender) < 0 &&
this.kvcomparator.compareRows(contender, this.targetkey) <= 0);
@ -123,7 +125,7 @@ class GetClosestRowBeforeTracker {
* @param kv
* @return true is the specified KeyValue is deleted, false if not
*/
private boolean isDeleted(final KeyValue kv) {
private boolean isDeleted(final Cell kv) {
if (this.deletes.isEmpty()) return false;
NavigableSet<KeyValue> rowdeletes = this.deletes.get(kv);
if (rowdeletes == null || rowdeletes.isEmpty()) return false;
@ -137,7 +139,7 @@ class GetClosestRowBeforeTracker {
* @param ds
* @return True is the specified KeyValue is deleted, false if not
*/
public boolean isDeleted(final KeyValue kv, final NavigableSet<KeyValue> ds) {
public boolean isDeleted(final Cell kv, final NavigableSet<KeyValue> ds) {
if (deletes == null || deletes.isEmpty()) return false;
for (KeyValue d: ds) {
long kvts = kv.getTimestamp();
@ -178,7 +180,7 @@ class GetClosestRowBeforeTracker {
* @param kv
* @return True if we removed <code>k</code> from <code>candidates</code>.
*/
boolean handleDeletes(final KeyValue kv) {
boolean handleDeletes(final Cell kv) {
addDelete(kv);
boolean deleted = false;
if (!hasCandidate()) return deleted;
@ -194,8 +196,8 @@ class GetClosestRowBeforeTracker {
* @param kv
* @return True if we added a candidate
*/
boolean handle(final KeyValue kv) {
if (kv.isDelete()) {
boolean handle(final Cell kv) {
if (KeyValueUtil.ensureKeyValue(kv).isDelete()) {
handleDeletes(kv);
return false;
}
@ -212,7 +214,7 @@ class GetClosestRowBeforeTracker {
/**
* @return Best candidate or null.
*/
public KeyValue getCandidate() {
public Cell getCandidate() {
return this.candidate;
}
@ -225,11 +227,11 @@ class GetClosestRowBeforeTracker {
* @param firstOnRow on row kv.
* @return True if we went too far, past the target key.
*/
boolean isTooFar(final KeyValue kv, final KeyValue firstOnRow) {
boolean isTooFar(final Cell kv, final Cell firstOnRow) {
return this.kvcomparator.compareRows(kv, firstOnRow) > 0;
}
boolean isTargetTable(final KeyValue kv) {
boolean isTargetTable(final Cell kv) {
if (!metaregion) return true;
// Compare start of keys row. Compare including delimiter. Saves having
// to calculate where tablename ends in the candidate kv.

View File

@ -102,7 +102,6 @@ import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterWrapper;
import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
import org.apache.hadoop.hbase.io.HeapSize;
@ -3214,7 +3213,7 @@ public class HRegion implements HeapSize { // , Writable{
for (KeyValue kv: val.getKeyValues()) {
// Check this edit is for me. Also, guard against writing the special
// METACOLUMN info such as HBASE::CACHEFLUSH entries
if (kv.matchingFamily(WALEdit.METAFAMILY) ||
if (CellUtil.matchingFamily(kv, WALEdit.METAFAMILY) ||
!Bytes.equals(key.getEncodedRegionName(),
this.getRegionInfo().getEncodedNameAsBytes())) {
//this is a special edit, we should handle it
@ -3228,7 +3227,7 @@ public class HRegion implements HeapSize { // , Writable{
continue;
}
// Figure which store the edit is meant for.
if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
if (store == null || !CellUtil.matchingFamily(kv, store.getFamily().getName())) {
store = this.stores.get(kv.getFamily());
}
if (store == null) {
@ -3632,7 +3631,7 @@ public class HRegion implements HeapSize { // , Writable{
/**
* If the joined heap data gathering is interrupted due to scan limits, this will
* contain the row for which we are populating the values.*/
protected KeyValue joinedContinuationRow = null;
protected Cell joinedContinuationRow = null;
// KeyValue indicating that limit is reached when scanning
private final KeyValue KV_LIMIT = new KeyValue();
protected final byte[] stopRow;
@ -3795,7 +3794,7 @@ public class HRegion implements HeapSize { // , Writable{
private void populateFromJoinedHeap(List<Cell> results, int limit)
throws IOException {
assert joinedContinuationRow != null;
KeyValue kv = populateResult(results, this.joinedHeap, limit,
Cell kv = populateResult(results, this.joinedHeap, limit,
joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
joinedContinuationRow.getRowLength());
if (kv != KV_LIMIT) {
@ -3817,16 +3816,16 @@ public class HRegion implements HeapSize { // , Writable{
* @param length length for currentRow
* @return KV_LIMIT if limit reached, next KeyValue otherwise.
*/
private KeyValue populateResult(List<Cell> results, KeyValueHeap heap, int limit,
private Cell populateResult(List<Cell> results, KeyValueHeap heap, int limit,
byte[] currentRow, int offset, short length) throws IOException {
KeyValue nextKv;
Cell nextKv;
do {
heap.next(results, limit - results.size());
if (limit > 0 && results.size() == limit) {
return KV_LIMIT;
}
nextKv = heap.peek();
} while (nextKv != null && nextKv.matchingRow(currentRow, offset, length));
} while (nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length));
return nextKv;
}
@ -3870,7 +3869,7 @@ public class HRegion implements HeapSize { // , Writable{
}
// Let's see what we have in the storeHeap.
KeyValue current = this.storeHeap.peek();
Cell current = this.storeHeap.peek();
byte[] currentRow = null;
int offset = 0;
@ -3901,7 +3900,7 @@ public class HRegion implements HeapSize { // , Writable{
continue;
}
KeyValue nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
Cell nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
length);
// Ok, we are good, let's try to get some results from the main heap.
if (nextKv == KV_LIMIT) {
@ -3940,14 +3939,14 @@ public class HRegion implements HeapSize { // , Writable{
// These values are not needed for filter to work, so we postpone their
// fetch to (possibly) reduce amount of data loads from disk.
if (this.joinedHeap != null) {
KeyValue nextJoinedKv = joinedHeap.peek();
Cell nextJoinedKv = joinedHeap.peek();
// If joinedHeap is pointing to some other row, try to seek to a correct one.
boolean mayHaveData =
(nextJoinedKv != null && nextJoinedKv.matchingRow(currentRow, offset, length))
|| (this.joinedHeap.requestSeek(KeyValue.createFirstOnRow(currentRow, offset, length),
true, true)
&& joinedHeap.peek() != null
&& joinedHeap.peek().matchingRow(currentRow, offset, length));
boolean mayHaveData = (nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv,
currentRow, offset, length))
|| (this.joinedHeap.requestSeek(
KeyValueUtil.createFirstOnRow(currentRow, offset, length), true, true)
&& joinedHeap.peek() != null && CellUtil.matchingRow(joinedHeap.peek(),
currentRow, offset, length));
if (mayHaveData) {
joinedContinuationRow = current;
populateFromJoinedHeap(results, limit);
@ -3999,9 +3998,9 @@ public class HRegion implements HeapSize { // , Writable{
protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException {
assert this.joinedContinuationRow == null: "Trying to go to next row during joinedHeap read.";
KeyValue next;
Cell next;
while ((next = this.storeHeap.peek()) != null &&
next.matchingRow(currentRow, offset, length)) {
CellUtil.matchingRow(next, currentRow, offset, length)) {
this.storeHeap.next(MOCKED_LIST);
}
resetFilters();
@ -4045,7 +4044,7 @@ public class HRegion implements HeapSize { // , Writable{
boolean result = false;
startRegionOperation();
try {
KeyValue kv = KeyValue.createFirstOnRow(row);
KeyValue kv = KeyValueUtil.createFirstOnRow(row);
// use request seek to make use of the lazy seek option. See HBASE-5520
result = this.storeHeap.requestSeek(kv, true, true);
if (this.joinedHeap != null) {

View File

@ -20,8 +20,8 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.FileNotFoundException;
import java.io.InterruptedIOException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.Reference;
@ -534,7 +535,7 @@ public class HRegionFileSystem {
// If it is outside the range, return directly.
if (top) {
//check if larger than last key.
KeyValue splitKey = KeyValue.createFirstOnRow(splitRow);
KeyValue splitKey = KeyValueUtil.createFirstOnRow(splitRow);
byte[] lastKey = f.createReader().getLastKey();
// If lastKey is null means storefile is empty.
if (lastKey == null) return null;
@ -544,7 +545,7 @@ public class HRegionFileSystem {
}
} else {
//check if smaller than first key
KeyValue splitKey = KeyValue.createLastOnRow(splitRow);
KeyValue splitKey = KeyValueUtil.createLastOnRow(splitRow);
byte[] firstKey = f.createReader().getFirstKey();
// If firstKey is null means storefile is empty.
if (firstKey == null) return null;

View File

@ -47,11 +47,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Scan;
@ -631,19 +633,19 @@ public class HStore implements Store {
}
if (verifyBulkLoads) {
KeyValue prevKV = null;
Cell prevKV = null;
HFileScanner scanner = reader.getScanner(false, false, false);
scanner.seekTo();
do {
KeyValue kv = scanner.getKeyValue();
Cell kv = scanner.getKeyValue();
if (prevKV != null) {
if (Bytes.compareTo(prevKV.getRowArray(), prevKV.getRowOffset(),
prevKV.getRowLength(), kv.getRowArray(), kv.getRowOffset(),
kv.getRowLength()) > 0) {
throw new InvalidHFileException("Previous row is greater than"
+ " current row: path=" + srcPath + " previous="
+ Bytes.toStringBinary(prevKV.getKey()) + " current="
+ Bytes.toStringBinary(kv.getKey()));
+ Bytes.toStringBinary(KeyValueUtil.ensureKeyValue(prevKV).getKey()) + " current="
+ Bytes.toStringBinary(KeyValueUtil.ensureKeyValue(kv).getKey()));
}
if (Bytes.compareTo(prevKV.getFamilyArray(), prevKV.getFamilyOffset(),
prevKV.getFamilyLength(), kv.getFamilyArray(), kv.getFamilyOffset(),
@ -1527,7 +1529,7 @@ public class HStore implements Store {
return wantedVersions > maxVersions ? maxVersions: wantedVersions;
}
static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
static boolean isExpired(final Cell key, final long oldestTimestamp) {
return key.getTimestamp() < oldestTimestamp;
}
@ -1557,15 +1559,17 @@ public class HStore implements Store {
StoreFile sf = sfIterator.next();
sfIterator.remove(); // Remove sf from iterator.
boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
KeyValue keyv = state.getCandidate();
KeyValue keyv = KeyValueUtil.ensureKeyValue(state.getCandidate());
// we have an optimization here which stops the search if we find exact match.
if (keyv != null && keyv.matchingRow(row)) return state.getCandidate();
if (keyv != null && CellUtil.matchingRow(keyv, row)) {
return KeyValueUtil.ensureKeyValue(state.getCandidate());
}
if (haveNewCandidate) {
sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
sfIterator, state.getTargetKey(), state.getCandidate());
sfIterator, state.getTargetKey(), KeyValueUtil.ensureKeyValue(state.getCandidate()));
}
}
return state.getCandidate();
return KeyValueUtil.ensureKeyValue(state.getCandidate());
} finally {
this.lock.readLock().unlock();
}
@ -1615,7 +1619,7 @@ public class HStore implements Store {
// If here, need to start backing up.
while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
firstOnRow.getKeyLength())) {
KeyValue kv = scanner.getKeyValue();
Cell kv = scanner.getKeyValue();
if (!state.isTargetTable(kv)) break;
if (!state.isBetterCandidate(kv)) break;
// Make new first on row.
@ -1663,7 +1667,7 @@ public class HStore implements Store {
throws IOException {
boolean foundCandidate = false;
do {
KeyValue kv = scanner.getKeyValue();
Cell kv = scanner.getKeyValue();
// If we are not in the row, skip.
if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
// Did we go beyond the target row? If so break.

View File

@ -26,7 +26,6 @@ import java.util.PriorityQueue;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
/**
@ -94,19 +93,19 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
}
}
public KeyValue peek() {
public Cell peek() {
if (this.current == null) {
return null;
}
return this.current.peek();
}
public KeyValue next() throws IOException {
public Cell next() throws IOException {
if(this.current == null) {
return null;
}
KeyValue kvReturn = this.current.next();
KeyValue kvNext = this.current.peek();
Cell kvReturn = this.current.next();
Cell kvNext = this.current.peek();
if (kvNext == null) {
this.current.close();
this.current = pollRealKV();
@ -138,7 +137,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
}
InternalScanner currentAsInternal = (InternalScanner)this.current;
boolean mayContainMoreRows = currentAsInternal.next(result, limit);
KeyValue pee = this.current.peek();
Cell pee = this.current.peek();
/*
* By definition, any InternalScanner must return false only when it has no
* further rows to be fetched. So, we can close a scanner if it returns
@ -202,7 +201,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
* @param right
* @return less than 0 if left is smaller, 0 if equal etc..
*/
public int compare(KeyValue left, KeyValue right) {
public int compare(Cell left, Cell right) {
return this.kvComparator.compare(left, right);
}
/**
@ -233,15 +232,15 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
* As individual scanners may run past their ends, those scanners are
* automatically closed and removed from the heap.
* <p>
* This function (and {@link #reseek(KeyValue)}) does not do multi-column
* This function (and {@link #reseek(Cell)}) does not do multi-column
* Bloom filter and lazy-seek optimizations. To enable those, call
* {@link #requestSeek(KeyValue, boolean, boolean)}.
* {@link #requestSeek(Cell, boolean, boolean)}.
* @param seekKey KeyValue to seek at or after
* @return true if KeyValues exist at or after specified key, false if not
* @throws IOException
*/
@Override
public boolean seek(KeyValue seekKey) throws IOException {
public boolean seek(Cell seekKey) throws IOException {
return generalizedSeek(false, // This is not a lazy seek
seekKey,
false, // forward (false: this is not a reseek)
@ -249,11 +248,11 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
}
/**
* This function is identical to the {@link #seek(KeyValue)} function except
* This function is identical to the {@link #seek(Cell)} function except
* that scanner.seek(seekKey) is changed to scanner.reseek(seekKey).
*/
@Override
public boolean reseek(KeyValue seekKey) throws IOException {
public boolean reseek(Cell seekKey) throws IOException {
return generalizedSeek(false, // This is not a lazy seek
seekKey,
true, // forward (true because this is reseek)
@ -264,7 +263,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
* {@inheritDoc}
*/
@Override
public boolean requestSeek(KeyValue key, boolean forward,
public boolean requestSeek(Cell key, boolean forward,
boolean useBloom) throws IOException {
return generalizedSeek(true, key, forward, useBloom);
}
@ -277,7 +276,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
* @param forward whether to seek forward (also known as reseek)
* @param useBloom whether to optimize seeks using Bloom filters
*/
private boolean generalizedSeek(boolean isLazy, KeyValue seekKey,
private boolean generalizedSeek(boolean isLazy, Cell seekKey,
boolean forward, boolean useBloom) throws IOException {
if (!isLazy && useBloom) {
throw new IllegalArgumentException("Multi-column Bloom filter " +
@ -292,7 +291,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
KeyValueScanner scanner;
while ((scanner = heap.poll()) != null) {
KeyValue topKey = scanner.peek();
Cell topKey = scanner.peek();
if (comparator.getComparator().compare(seekKey, topKey) <= 0) {
// Top KeyValue is at-or-after Seek KeyValue. We only know that all
// scanners are at or after seekKey (because fake keys of
@ -345,7 +344,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
while (kvScanner != null && !kvScanner.realSeekDone()) {
if (kvScanner.peek() != null) {
kvScanner.enforceSeek();
KeyValue curKV = kvScanner.peek();
Cell curKV = kvScanner.peek();
if (curKV != null) {
KeyValueScanner nextEarliestScanner = heap.peek();
if (nextEarliestScanner == null) {
@ -355,7 +354,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
// Compare the current scanner to the next scanner. We try to avoid
// putting the current one back into the heap if possible.
KeyValue nextKV = nextEarliestScanner.peek();
Cell nextKV = nextEarliestScanner.peek();
if (nextKV == null || comparator.compare(curKV, nextKV) < 0) {
// We already have the scanner with the earliest KV, so return it.
return kvScanner;

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Scan;
/**
@ -31,23 +31,23 @@ import org.apache.hadoop.hbase.client.Scan;
@InterfaceAudience.Private
public interface KeyValueScanner {
/**
* Look at the next KeyValue in this scanner, but do not iterate scanner.
* @return the next KeyValue
* Look at the next Cell in this scanner, but do not iterate scanner.
* @return the next Cell
*/
KeyValue peek();
Cell peek();
/**
* Return the next KeyValue in this scanner, iterating the scanner
* @return the next KeyValue
* Return the next Cell in this scanner, iterating the scanner
* @return the next Cell
*/
KeyValue next() throws IOException;
Cell next() throws IOException;
/**
* Seek the scanner at or after the specified KeyValue.
* @param key seek value
* @return true if scanner has values left, false if end of scanner
*/
boolean seek(KeyValue key) throws IOException;
boolean seek(Cell key) throws IOException;
/**
* Reseek the scanner at or after the specified KeyValue.
@ -57,7 +57,7 @@ public interface KeyValueScanner {
* @param key seek value (should be non-null)
* @return true if scanner has values left, false if end of scanner
*/
boolean reseek(KeyValue key) throws IOException;
boolean reseek(Cell key) throws IOException;
/**
* Get the sequence id associated with this KeyValueScanner. This is required
@ -98,7 +98,7 @@ public interface KeyValueScanner {
* @param forward do a forward-only "reseek" instead of a random-access seek
* @param useBloom whether to enable multi-column Bloom filter optimization
*/
boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
throws IOException;
/**
@ -126,10 +126,10 @@ public interface KeyValueScanner {
// Support for "Reversed Scanner"
/**
* Seek the scanner at or before the row of specified KeyValue, it firstly
* tries to seek the scanner at or after the specified KeyValue, return if
* peek KeyValue of scanner has the same row with specified KeyValue,
* otherwise seek the scanner at the first KeyValue of the row which is the
* Seek the scanner at or before the row of specified Cell, it firstly
* tries to seek the scanner at or after the specified Cell, return if
* peek KeyValue of scanner has the same row with specified Cell,
* otherwise seek the scanner at the first Cell of the row which is the
* previous row of specified KeyValue
*
* @param key seek KeyValue
@ -137,16 +137,16 @@ public interface KeyValueScanner {
* KeyValue does not exist
*
*/
public boolean backwardSeek(KeyValue key) throws IOException;
public boolean backwardSeek(Cell key) throws IOException;
/**
* Seek the scanner at the first KeyValue of the row which is the previous row
* Seek the scanner at the first Cell of the row which is the previous row
* of specified key
* @param key seek value
* @return true if the scanner at the first valid KeyValue of previous row,
* false if not existing such KeyValue
* @return true if the scanner at the first valid Cell of previous row,
* false if not existing such Cell
*/
public boolean seekToPreviousRow(KeyValue key) throws IOException;
public boolean seekToPreviousRow(Cell key) throws IOException;
/**
* Seek the scanner at the first KeyValue of last row

View File

@ -23,7 +23,7 @@ import java.util.SortedSet;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Scan;
/**
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.Scan;
public abstract class NonLazyKeyValueScanner implements KeyValueScanner {
@Override
public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
public boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
throws IOException {
return doRealSeek(this, kv, forward);
}
@ -51,7 +51,7 @@ public abstract class NonLazyKeyValueScanner implements KeyValueScanner {
}
public static boolean doRealSeek(KeyValueScanner scanner,
KeyValue kv, boolean forward) throws IOException {
Cell kv, boolean forward) throws IOException {
return forward ? scanner.reseek(kv) : scanner.seek(kv);
}

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
/**
* A "non-reversed & non-lazy" scanner which does not support backward scanning
@ -34,13 +34,13 @@ public abstract class NonReversedNonLazyKeyValueScanner extends
NonLazyKeyValueScanner {
@Override
public boolean backwardSeek(KeyValue key) throws IOException {
public boolean backwardSeek(Cell key) throws IOException {
throw new NotImplementedException("backwardSeek must not be called on a "
+ "non-reversed scanner");
}
@Override
public boolean seekToPreviousRow(KeyValue key) throws IOException {
public boolean seekToPreviousRow(Cell key) throws IOException {
throw new NotImplementedException("seekToPreviousRow must not be called on a "
+ "non-reversed scanner");
}

View File

@ -23,7 +23,8 @@ import java.util.List;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
/**
@ -47,26 +48,26 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
}
@Override
public boolean seek(KeyValue seekKey) throws IOException {
public boolean seek(Cell seekKey) throws IOException {
throw new IllegalStateException(
"seek cannot be called on ReversedKeyValueHeap");
}
@Override
public boolean reseek(KeyValue seekKey) throws IOException {
public boolean reseek(Cell seekKey) throws IOException {
throw new IllegalStateException(
"reseek cannot be called on ReversedKeyValueHeap");
}
@Override
public boolean requestSeek(KeyValue key, boolean forward, boolean useBloom)
public boolean requestSeek(Cell key, boolean forward, boolean useBloom)
throws IOException {
throw new IllegalStateException(
"requestSeek cannot be called on ReversedKeyValueHeap");
}
@Override
public boolean seekToPreviousRow(KeyValue seekKey) throws IOException {
public boolean seekToPreviousRow(Cell seekKey) throws IOException {
if (current == null) {
return false;
}
@ -75,7 +76,7 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
KeyValueScanner scanner;
while ((scanner = heap.poll()) != null) {
KeyValue topKey = scanner.peek();
Cell topKey = scanner.peek();
if (comparator.getComparator().compareRows(topKey.getRowArray(),
topKey.getRowOffset(), topKey.getRowLength(), seekKey.getRowArray(),
seekKey.getRowOffset(), seekKey.getRowLength()) < 0) {
@ -97,7 +98,7 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
}
@Override
public boolean backwardSeek(KeyValue seekKey) throws IOException {
public boolean backwardSeek(Cell seekKey) throws IOException {
if (current == null) {
return false;
}
@ -106,8 +107,8 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
KeyValueScanner scanner;
while ((scanner = heap.poll()) != null) {
KeyValue topKey = scanner.peek();
if ((comparator.getComparator().matchingRows(seekKey, topKey) && comparator
Cell topKey = scanner.peek();
if ((CellUtil.matchingRow(seekKey, topKey) && comparator
.getComparator().compare(seekKey, topKey) <= 0)
|| comparator.getComparator().compareRows(seekKey, topKey) > 0) {
heap.add(scanner);
@ -124,12 +125,12 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
}
@Override
public KeyValue next() throws IOException {
public Cell next() throws IOException {
if (this.current == null) {
return null;
}
KeyValue kvReturn = this.current.next();
KeyValue kvNext = this.current.peek();
Cell kvReturn = this.current.next();
Cell kvNext = this.current.peek();
if (kvNext == null
|| this.comparator.kvComparator.compareRows(kvNext, kvReturn) > 0) {
if (this.current.seekToPreviousRow(kvReturn)) {
@ -180,7 +181,7 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
* @param right
* @return less than 0 if left is smaller, 0 if equal etc..
*/
public int compareRows(KeyValue left, KeyValue right) {
public int compareRows(Cell left, Cell right) {
return super.kvComparator.compareRows(left, right);
}
}

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
@ -68,7 +68,7 @@ class ReversedRegionScannerImpl extends RegionScannerImpl {
assert super.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read.";
byte row[] = new byte[length];
System.arraycopy(currentRow, offset, row, 0, length);
this.storeHeap.seekToPreviousRow(KeyValue.createFirstOnRow(row));
this.storeHeap.seekToPreviousRow(KeyValueUtil.createFirstOnRow(row));
resetFilters();
// Calling the hook in CP which allows it to do a fast forward
if (this.region.getCoprocessorHost() != null) {

View File

@ -23,6 +23,8 @@ import java.util.List;
import java.util.NavigableSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
@ -68,11 +70,11 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
@Override
protected void seekScanners(List<? extends KeyValueScanner> scanners,
KeyValue seekKey, boolean isLazy, boolean isParallelSeek)
Cell seekKey, boolean isLazy, boolean isParallelSeek)
throws IOException {
// Seek all scanners to the start of the Row (or if the exact matching row
// key does not exist, then to the start of the previous matching Row).
if (seekKey.matchingRow(HConstants.EMPTY_START_ROW)) {
if (CellUtil.matchingRow(seekKey, HConstants.EMPTY_START_ROW)) {
for (KeyValueScanner scanner : scanners) {
scanner.seekToLastRow();
}
@ -84,7 +86,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
}
@Override
protected boolean seekToNextRow(KeyValue kv) throws IOException {
protected boolean seekToNextRow(Cell kv) throws IOException {
return seekToPreviousRow(kv);
}
@ -97,7 +99,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
}
@Override
protected void checkScanOrder(KeyValue prevKV, KeyValue kv,
protected void checkScanOrder(Cell prevKV, Cell kv,
KeyValue.KVComparator comparator) throws IOException {
// Check that the heap gives us KVs in an increasing order for same row and
// decreasing order for different rows.
@ -109,19 +111,19 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
}
@Override
public boolean reseek(KeyValue kv) throws IOException {
public boolean reseek(Cell kv) throws IOException {
throw new IllegalStateException(
"reseek cannot be called on ReversedStoreScanner");
}
@Override
public boolean seek(KeyValue key) throws IOException {
public boolean seek(Cell key) throws IOException {
throw new IllegalStateException(
"seek cannot be called on ReversedStoreScanner");
}
@Override
public boolean seekToPreviousRow(KeyValue key) throws IOException {
public boolean seekToPreviousRow(Cell key) throws IOException {
lock.lock();
try {
checkReseek();
@ -133,7 +135,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
}
@Override
public boolean backwardSeek(KeyValue key) throws IOException {
public boolean backwardSeek(Cell key) throws IOException {
lock.lock();
try {
checkReseek();

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
@ -151,7 +152,7 @@ public class ScanQueryMatcher {
this.rowComparator = scanInfo.getComparator();
this.deletes = new ScanDeleteTracker();
this.stopRow = scan.getStopRow();
this.startKey = KeyValue.createFirstDeleteFamilyOnRow(scan.getStartRow(),
this.startKey = KeyValueUtil.createFirstDeleteFamilyOnRow(scan.getStartRow(),
scanInfo.getFamily());
this.filter = scan.getFilter();
this.earliestPutTs = earliestPutTs;
@ -535,12 +536,12 @@ public class ScanQueryMatcher {
public KeyValue getKeyForNextColumn(KeyValue kv) {
ColumnCount nextColumn = columns.getColumnHint();
if (nextColumn == null) {
return KeyValue.createLastOnRow(
return KeyValueUtil.createLastOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
} else {
return KeyValue.createFirstOnRow(
return KeyValueUtil.createFirstOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength());
@ -548,7 +549,7 @@ public class ScanQueryMatcher {
}
public KeyValue getKeyForNextRow(KeyValue kv) {
return KeyValue.createLastOnRow(
return KeyValueUtil.createLastOnRow(
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
null, 0, 0,
null, 0, 0);

View File

@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
@ -1311,11 +1311,11 @@ public class StoreFile {
&& Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
return true;
}
KeyValue smallestScanKeyValue = scan.isReversed() ? KeyValue
.createFirstOnRow(scan.getStopRow()) : KeyValue.createFirstOnRow(scan
KeyValue smallestScanKeyValue = scan.isReversed() ? KeyValueUtil
.createFirstOnRow(scan.getStopRow()) : KeyValueUtil.createFirstOnRow(scan
.getStartRow());
KeyValue largestScanKeyValue = scan.isReversed() ? KeyValue
.createLastOnRow(scan.getStartRow()) : KeyValue.createLastOnRow(scan
KeyValue largestScanKeyValue = scan.isReversed() ? KeyValueUtil
.createLastOnRow(scan.getStartRow()) : KeyValueUtil.createLastOnRow(scan
.getStopRow());
boolean nonOverLapping = (getComparator().compareFlatKey(
this.getFirstKey(), largestScanKeyValue.getKey()) > 0 && !Bytes

View File

@ -29,8 +29,11 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
@ -47,11 +50,11 @@ public class StoreFileScanner implements KeyValueScanner {
// the reader it comes from:
private final StoreFile.Reader reader;
private final HFileScanner hfs;
private KeyValue cur = null;
private Cell cur = null;
private boolean realSeekDone;
private boolean delayedReseek;
private KeyValue delayedSeekKV;
private Cell delayedSeekKV;
private boolean enforceMVCC = false;
private boolean hasMVCCInfo = false;
@ -124,12 +127,12 @@ public class StoreFileScanner implements KeyValueScanner {
return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]";
}
public KeyValue peek() {
public Cell peek() {
return cur;
}
public KeyValue next() throws IOException {
KeyValue retKey = cur;
public Cell next() throws IOException {
Cell retKey = cur;
try {
// only seek if we aren't at the end. cur == null implies 'end'.
@ -145,7 +148,7 @@ public class StoreFileScanner implements KeyValueScanner {
return retKey;
}
public boolean seek(KeyValue key) throws IOException {
public boolean seek(Cell key) throws IOException {
if (seekCount != null) seekCount.incrementAndGet();
try {
@ -166,7 +169,7 @@ public class StoreFileScanner implements KeyValueScanner {
}
}
public boolean reseek(KeyValue key) throws IOException {
public boolean reseek(Cell key) throws IOException {
if (seekCount != null) seekCount.incrementAndGet();
try {
@ -190,7 +193,7 @@ public class StoreFileScanner implements KeyValueScanner {
protected boolean skipKVsNewerThanReadpoint() throws IOException {
// We want to ignore all key-values that are newer than our current
// readPoint
KeyValue startKV = cur;
Cell startKV = cur;
while(enforceMVCC
&& cur != null
&& (cur.getMvccVersion() > readPt)) {
@ -216,7 +219,7 @@ public class StoreFileScanner implements KeyValueScanner {
// not old enough during flush). Make sure that we set it correctly now,
// so that the comparision order does not change.
if (cur.getMvccVersion() <= readPt) {
cur.setMvccVersion(0);
KeyValueUtil.ensureKeyValue(cur).setMvccVersion(0);
}
return true;
}
@ -233,7 +236,7 @@ public class StoreFileScanner implements KeyValueScanner {
* @return false if not found or if k is after the end.
* @throws IOException
*/
public static boolean seekAtOrAfter(HFileScanner s, KeyValue k)
public static boolean seekAtOrAfter(HFileScanner s, Cell k)
throws IOException {
int result = s.seekTo(k);
if(result < 0) {
@ -252,7 +255,7 @@ public class StoreFileScanner implements KeyValueScanner {
return true;
}
static boolean reseekAtOrAfter(HFileScanner s, KeyValue k)
static boolean reseekAtOrAfter(HFileScanner s, Cell k)
throws IOException {
//This function is similar to seekAtOrAfter function
int result = s.reseekTo(k);
@ -294,7 +297,7 @@ public class StoreFileScanner implements KeyValueScanner {
* row/column and use OLDEST_TIMESTAMP in the seek key.
*/
@Override
public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
public boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
throws IOException {
if (kv.getFamilyLength() == 0) {
useBloom = false;
@ -308,7 +311,7 @@ public class StoreFileScanner implements KeyValueScanner {
kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
kv.getQualifierOffset(), kv.getQualifierLength());
} else if (this.matcher != null && !matcher.hasNullColumnInQuery() &&
(kv.isDeleteFamily() || kv.isDeleteFamilyVersion())) {
((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) {
// if there is no such delete family kv in the store file,
// then no need to seek.
haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(),
@ -332,7 +335,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).
cur = kv.createFirstOnRowColTS(maxTimestampInFile);
cur = KeyValueUtil.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
@ -350,7 +353,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.
cur = kv.createLastOnRowCol();
cur = KeyValueUtil.createLastOnRowCol(kv);
realSeekDone = true;
return true;
@ -402,18 +405,19 @@ public class StoreFileScanner implements KeyValueScanner {
}
@Override
public boolean seekToPreviousRow(KeyValue key) throws IOException {
public boolean seekToPreviousRow(Cell key) throws IOException {
try {
try {
KeyValue seekKey = KeyValue.createFirstOnRow(key.getRow());
KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
key.getRowLength());
if (seekCount != null) seekCount.incrementAndGet();
if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(),
seekKey.getKeyLength())) {
close();
return false;
}
KeyValue firstKeyOfPreviousRow = KeyValue.createFirstOnRow(hfs
.getKeyValue().getRow());
KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue()
.getRowArray(), hfs.getKeyValue().getRowOffset(), hfs.getKeyValue().getRowLength());
if (seekCount != null) seekCount.incrementAndGet();
if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
@ -430,10 +434,7 @@ public class StoreFileScanner implements KeyValueScanner {
this.stopSkippingKVsIfNextRow = false;
}
if (!resultOfSkipKVs
|| Bytes.compareTo(cur.getBuffer(), cur.getRowOffset(),
cur.getRowLength(), firstKeyOfPreviousRow.getBuffer(),
firstKeyOfPreviousRow.getRowOffset(),
firstKeyOfPreviousRow.getRowLength()) > 0) {
|| KeyValue.COMPARATOR.compareRows(cur, firstKeyOfPreviousRow) > 0) {
return seekToPreviousRow(firstKeyOfPreviousRow);
}
@ -453,7 +454,7 @@ public class StoreFileScanner implements KeyValueScanner {
if (lastRow == null) {
return false;
}
KeyValue seekKey = KeyValue.createFirstOnRow(lastRow);
KeyValue seekKey = KeyValueUtil.createFirstOnRow(lastRow);
if (seek(seekKey)) {
return true;
} else {
@ -462,7 +463,7 @@ public class StoreFileScanner implements KeyValueScanner {
}
@Override
public boolean backwardSeek(KeyValue key) throws IOException {
public boolean backwardSeek(Cell key) throws IOException {
seek(key);
if (cur == null
|| Bytes.compareTo(cur.getRowArray(), cur.getRowOffset(),

View File

@ -82,7 +82,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* KVs skipped via seeking to next row/column. TODO: estimate them?
*/
private long kvsScanned = 0;
private KeyValue prevKV = null;
private Cell prevKV = null;
/** We don't ever expect to change this, the constant is just for clarity. */
static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;
@ -94,7 +94,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
LAZY_SEEK_ENABLED_BY_DEFAULT;
// if heap == null and lastTop != null, you need to reseek given the key below
protected KeyValue lastTop = null;
protected Cell lastTop = null;
// A flag whether use pread for scan
private boolean scanUsePread = false;
@ -301,7 +301,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @throws IOException
*/
protected void seekScanners(List<? extends KeyValueScanner> scanners,
KeyValue seekKey, boolean isLazy, boolean isParallelSeek)
Cell seekKey, boolean isLazy, boolean isParallelSeek)
throws IOException {
// Seek all scanners to the start of the Row (or if the exact matching row
// key does not exist, then to the start of the next matching Row).
@ -368,7 +368,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
@Override
public KeyValue peek() {
public Cell peek() {
lock.lock();
try {
if (this.heap == null) {
@ -405,7 +405,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
@Override
public boolean seek(KeyValue key) throws IOException {
public boolean seek(Cell key) throws IOException {
lock.lock();
try {
// reset matcher state, in case that underlying store changed
@ -437,7 +437,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
return false;
}
KeyValue peeked = this.heap.peek();
Cell peeked = this.heap.peek();
if (peeked == null) {
close();
return false;
@ -454,7 +454,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
matcher.setRow(row, offset, length);
}
KeyValue kv;
Cell kv;
// Only do a sanity-check if store and comparator are available.
KeyValue.KVComparator comparator =
@ -466,7 +466,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
checkScanOrder(prevKV, kv, comparator);
prevKV = kv;
ScanQueryMatcher.MatchCode qcode = matcher.match(kv);
ScanQueryMatcher.MatchCode qcode = matcher.match(KeyValueUtil.ensureKeyValue(kv));
switch(qcode) {
case INCLUDE:
case INCLUDE_AND_SEEK_NEXT_ROW:
@ -482,7 +482,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
if (storeLimit > -1 &&
this.countPerRow > (storeLimit + storeOffset)) {
// do what SEEK_NEXT_ROW does.
if (!matcher.moreRowsMayExistAfter(kv)) {
if (!matcher.moreRowsMayExistAfter(KeyValueUtil.ensureKeyValue(kv))) {
return false;
}
seekToNextRow(kv);
@ -497,12 +497,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
if (!matcher.moreRowsMayExistAfter(kv)) {
if (!matcher.moreRowsMayExistAfter(KeyValueUtil.ensureKeyValue(kv))) {
return false;
}
seekToNextRow(kv);
} else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
seekAsDirection(matcher.getKeyForNextColumn(kv));
seekAsDirection(matcher.getKeyForNextColumn(KeyValueUtil.ensureKeyValue(kv)));
} else {
this.heap.next();
}
@ -522,7 +522,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
case SEEK_NEXT_ROW:
// This is just a relatively simple end of scan fix, to short-cut end
// us if there is an endKey in the scan.
if (!matcher.moreRowsMayExistAfter(kv)) {
if (!matcher.moreRowsMayExistAfter(KeyValueUtil.ensureKeyValue(kv))) {
return false;
}
@ -530,7 +530,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
break;
case SEEK_NEXT_COL:
seekAsDirection(matcher.getKeyForNextColumn(kv));
seekAsDirection(matcher.getKeyForNextColumn(KeyValueUtil.ensureKeyValue(kv)));
break;
case SKIP:
@ -619,7 +619,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
return false;
}
protected void resetScannerStack(KeyValue lastTopKey) throws IOException {
protected void resetScannerStack(Cell lastTopKey) throws IOException {
if (heap != null) {
throw new RuntimeException("StoreScanner.reseek run on an existing heap!");
}
@ -638,7 +638,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// Reset the state of the Query Matcher and set to top row.
// Only reset and call setRow if the row changes; avoids confusing the
// query matcher if scanning intra-row.
KeyValue kv = heap.peek();
Cell kv = heap.peek();
if (kv == null) {
kv = lastTopKey;
}
@ -660,7 +660,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @param comparator
* @throws IOException
*/
protected void checkScanOrder(KeyValue prevKV, KeyValue kv,
protected void checkScanOrder(Cell prevKV, Cell kv,
KeyValue.KVComparator comparator) throws IOException {
// Check that the heap gives us KVs in an increasing order.
assert prevKV == null || comparator == null
@ -668,8 +668,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
+ " followed by a " + "smaller key " + kv + " in cf " + store;
}
protected boolean seekToNextRow(KeyValue kv) throws IOException {
return reseek(matcher.getKeyForNextRow(kv));
protected boolean seekToNextRow(Cell kv) throws IOException {
return reseek(KeyValueUtil.createLastOnRow(kv));
}
/**
@ -684,7 +684,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
@Override
public boolean reseek(KeyValue kv) throws IOException {
public boolean reseek(Cell kv) throws IOException {
lock.lock();
try {
//Heap will not be null, if this is called from next() which.
@ -712,7 +712,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @throws IOException
*/
private void parallelSeek(final List<? extends KeyValueScanner>
scanners, final KeyValue kv) throws IOException {
scanners, final Cell kv) throws IOException {
if (scanners.isEmpty()) return;
int storeFileScannerCount = scanners.size();
CountDownLatch latch = new CountDownLatch(storeFileScannerCount);

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;

View File

@ -24,11 +24,10 @@ import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
/**
* Handler to seek storefiles in parallel.
@ -37,12 +36,12 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
public class ParallelSeekHandler extends EventHandler {
private static final Log LOG = LogFactory.getLog(ParallelSeekHandler.class);
private KeyValueScanner scanner;
private KeyValue keyValue;
private Cell keyValue;
private long readPoint;
private CountDownLatch latch;
private Throwable err = null;
public ParallelSeekHandler(KeyValueScanner scanner,KeyValue keyValue,
public ParallelSeekHandler(KeyValueScanner scanner,Cell keyValue,
long readPoint, CountDownLatch latch) {
super(null, EventType.RS_PARALLEL_SEEK);
this.scanner = scanner;

View File

@ -1437,7 +1437,7 @@ public class HLogSplitter {
// We don't handle HBASE-2231 because we may or may not replay a compaction event.
// Details at https://issues.apache.org/jira/browse/HBASE-2231?focusedCommentId=13647143&
// page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13647143
if (kv.matchingFamily(WALEdit.METAFAMILY)) {
if (CellUtil.matchingFamily(kv, WALEdit.METAFAMILY)) {
skippedKVs.add(kv);
continue;
}

View File

@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.HeapSize;
@ -265,7 +266,7 @@ public class WALEdit implements Writable, HeapSize {
* @return deserialized CompactionDescriptor or null.
*/
public static CompactionDescriptor getCompaction(KeyValue kv) throws IOException {
if (kv.matchingRow(METAROW) && kv.matchingColumn(METAFAMILY, COMPACTION)) {
if (CellUtil.matchingRow(kv, METAROW) && CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
return CompactionDescriptor.parseFrom(kv.getValue());
}
return null;

View File

@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
@ -230,7 +231,7 @@ public class WALEditsReplaySink {
List<KeyValue> kvs = edit.getKeyValues();
for (KeyValue kv : kvs) {
// filtering HLog meta entries
if (kv.matchingFamily(WALEdit.METAFAMILY)) continue;
if (CellUtil.matchingFamily(kv, WALEdit.METAFAMILY)) continue;
setLocation(conn.locateRegion(tableName, kv.getRow()));
skip = true;

View File

@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.replication.regionserver;
import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
import java.io.IOException;
import java.util.List;
import java.util.NavigableMap;
@ -27,7 +31,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -35,17 +38,18 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
@ -56,9 +60,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.zookeeper.KeeperException;
import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
import static org.apache.hadoop.hbase.HConstants.REPLICATION_ENABLE_KEY;
import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* Gateway to Replication. Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
@ -246,7 +248,7 @@ public class Replication implements WALActionsListener,
for (KeyValue kv : logEdit.getKeyValues()) {
family = kv.getFamily();
// This is expected and the KV should not be replicated
if (kv.matchingFamily(WALEdit.METAFAMILY)) continue;
if (CellUtil.matchingFamily(kv, WALEdit.METAFAMILY)) continue;
// Unexpected, has a tendency to happen in unit tests
assert htd.getFamily(family) != null;

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
@ -672,7 +673,7 @@ public class ReplicationSource extends Thread
int distinctRowKeys = 1;
KeyValue lastKV = kvs.get(0);
for (int i = 0; i < edit.size(); i++) {
if (!kvs.get(i).matchingRow(lastKV)) {
if (!CellUtil.matchingRow(kvs.get(i), lastKV)) {
distinctRowKeys++;
}
}

View File

@ -25,6 +25,7 @@ import java.util.List;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
@ -99,14 +100,14 @@ public class CollectionBackedScanner extends NonReversedNonLazyKeyValueScanner {
}
@Override
public boolean seek(KeyValue seekKv) {
public boolean seek(Cell seekKv) {
// restart iterator
iter = data.iterator();
return reseek(seekKv);
}
@Override
public boolean reseek(KeyValue seekKv) {
public boolean reseek(Cell seekKv) {
while(iter.hasNext()){
KeyValue next = iter.next();
int ret = comparator.compare(next, seekKv);

View File

@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.util;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
@InterfaceAudience.Private
public class CompoundBloomFilterBase implements BloomFilterBase {
@ -84,7 +84,7 @@ public class CompoundBloomFilterBase implements BloomFilterBase {
// Make sure this does not specify a timestamp so that the default maximum
// (most recent) timestamp is used.
KeyValue kv = KeyValue.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0,
KeyValue kv = KeyValueUtil.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0,
qualifier, qoffset, qlength);
return kv.getKey();
}

View File

@ -30,8 +30,10 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
@ -121,7 +123,7 @@ public class TestHalfStoreFileReader {
final HFileScanner scanner = halfreader.getScanner(false, false);
scanner.seekTo();
KeyValue curr;
Cell curr;
do {
curr = scanner.getKeyValue();
KeyValue reseekKv =
@ -183,7 +185,7 @@ public class TestHalfStoreFileReader {
// Seek on the splitKey, should be in top, not in bottom
KeyValue foundKeyValue = doTestOfSeekBefore(p, fs, bottom, midKV, cacheConf);
Cell foundKeyValue = doTestOfSeekBefore(p, fs, bottom, midKV, cacheConf);
assertEquals(beforeMidKey, foundKeyValue);
// Seek tot the last thing should be the penultimate on the top, the one before the midkey on the bottom.
@ -213,7 +215,7 @@ public class TestHalfStoreFileReader {
assertNull(foundKeyValue);
}
private KeyValue doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, KeyValue seekBefore,
private Cell doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, KeyValue seekBefore,
CacheConfig cacheConfig)
throws IOException {
final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p,
@ -224,8 +226,8 @@ public class TestHalfStoreFileReader {
return scanner.getKeyValue();
}
private KeyValue getLastOnCol(KeyValue curr) {
return KeyValue.createLastOnRow(
private KeyValue getLastOnCol(Cell curr) {
return KeyValueUtil.createLastOnRow(
curr.getRowArray(), curr.getRowOffset(), curr.getRowLength(),
curr.getFamilyArray(), curr.getFamilyOffset(), curr.getFamilyLength(),
curr.getQualifierArray(), curr.getQualifierOffset(), curr.getQualifierLength());

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -252,7 +253,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 = midKv.createLastOnRowCol();
KeyValue lastMidKv =KeyValueUtil.createLastOnRowCol(midKv);
checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv);
}
}

View File

@ -35,7 +35,9 @@ import java.util.concurrent.ConcurrentSkipListSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec;
@ -113,14 +115,14 @@ public class TestPrefixTreeEncoding {
seeker.setCurrentBuffer(readBuffer);
// Seek before the first keyvalue;
KeyValue seekKey = KeyValue.createFirstDeleteFamilyOnRow(getRowKey(batchId, 0), CF_BYTES);
KeyValue seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(getRowKey(batchId, 0), CF_BYTES);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
.getKeyLength()), true);
assertEquals(null, seeker.getKeyValue());
// Seek before the middle keyvalue;
seekKey = KeyValue.createFirstDeleteFamilyOnRow(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3),
seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3),
CF_BYTES);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
@ -129,7 +131,7 @@ public class TestPrefixTreeEncoding {
assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3 - 1), seeker.getKeyValue().getRow());
// Seek before the last keyvalue;
seekKey = KeyValue.createFirstDeleteFamilyOnRow(Bytes.toBytes("zzzz"), CF_BYTES);
seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(Bytes.toBytes("zzzz"), CF_BYTES);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
.getKeyLength()), true);
@ -156,9 +158,9 @@ public class TestPrefixTreeEncoding {
ByteBuffer readBuffer = ByteBuffer.wrap(onDiskBytes, DataBlockEncoding.ID_SIZE,
onDiskBytes.length - DataBlockEncoding.ID_SIZE);
seeker.setCurrentBuffer(readBuffer);
KeyValue previousKV = null;
Cell previousKV = null;
do {
KeyValue currentKV = seeker.getKeyValue();
Cell currentKV = seeker.getKeyValue();
System.out.println(currentKV);
if (previousKV != null && KeyValue.COMPARATOR.compare(currentKV, previousKV) < 0) {
dumpInputKVSet();
@ -223,7 +225,7 @@ public class TestPrefixTreeEncoding {
for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
kvList.clear();
encodeSeeker.setCurrentBuffer(encodedData);
KeyValue firstOnRow = KeyValue.createFirstOnRow(getRowKey(batchId, i));
KeyValue firstOnRow = KeyValueUtil.createFirstOnRow(getRowKey(batchId, i));
encodeSeeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
firstOnRow.getKeyLength()), false);

View File

@ -23,8 +23,10 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -154,7 +156,7 @@ public class TestSeekToBlockWithEncoders {
Bytes.toBytes("q1"), Bytes.toBytes("val"));
sampleKv.add(kv4);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = KeyValue.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
KeyValue toSeek = KeyValueUtil.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
kv3.getRowLength(), null, 0, 0, null, 0, 0);
seekToTheKey(kv3, originalBuffer, toSeek);
}
@ -304,7 +306,7 @@ public class TestSeekToBlockWithEncoders {
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue
.getKeyLength()), false);
KeyValue keyValue2 = seeker.getKeyValue();
Cell keyValue2 = seeker.getKeyValue();
assertEquals(expected, keyValue2);
seeker.rewind();
}

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -540,7 +541,7 @@ public class TestHFileBlockIndex {
byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i);
// Key will be interpreted by KeyValue.KEY_COMPARATOR
byte[] k = KeyValue.createFirstOnRow(row, 0, row.length, row, 0, 0,
byte[] k = KeyValueUtil.createFirstOnRow(row, 0, row.length, row, 0, 0,
row, 0, 0).getKey();
byte[] v = TestHFileWriterV2.randomValue(rand);

View File

@ -36,9 +36,11 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.crypto.Cipher;
@ -220,8 +222,9 @@ public class TestHFileEncryption {
assertTrue("Initial seekTo failed", scanner.seekTo());
int i = 0;
do {
KeyValue kv = scanner.getKeyValue();
assertTrue("Read back an unexpected or invalid KV", testKvs.contains(kv));
Cell kv = scanner.getKeyValue();
assertTrue("Read back an unexpected or invalid KV",
testKvs.contains(KeyValueUtil.ensureKeyValue(kv)));
i++;
} while (scanner.next());
reader.close();

View File

@ -22,9 +22,11 @@ import java.io.IOException;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.util.Bytes;
@ -63,8 +65,8 @@ public class TestSeekTo extends HBaseTestCase {
}
}
}
static String toRowStr(KeyValue kv) {
return Bytes.toString(kv.getRow());
static String toRowStr(Cell kv) {
return Bytes.toString(KeyValueUtil.ensureKeyValue(kv).getRow());
}
Path makeNewFile(TagUsage tagUsage) throws IOException {

View File

@ -33,7 +33,9 @@ import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.Callable;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -196,7 +198,7 @@ public class TestHFileOutputFormat {
writer.write(new ImmutableBytesWritable(), kv);
assertFalse(original.equals(kv));
assertTrue(Bytes.equals(original.getRow(), kv.getRow()));
assertTrue(original.matchingColumn(kv.getFamily(), kv.getQualifier()));
assertTrue(CellUtil.matchingColumn(original, kv.getFamily(), kv.getQualifier()));
assertNotSame(original.getTimestamp(), kv.getTimestamp());
assertNotSame(HConstants.LATEST_TIMESTAMP, kv.getTimestamp());

View File

@ -180,7 +180,7 @@ public class DataBlockEncodingTool {
new ByteArrayOutputStream();
int j = 0;
while ((currentKV = scanner.next()) != null && j < kvLimit) {
while ((currentKV = KeyValueUtil.ensureKeyValue(scanner.next())) != null && j < kvLimit) {
// Iterates through key/value pairs
j++;
currentKey = currentKV.getKey();
@ -242,7 +242,7 @@ public class DataBlockEncodingTool {
}
int j = 0;
while ((currentKv = scanner.next()) != null && j < kvLimit) {
while ((currentKv = KeyValueUtil.ensureKeyValue(scanner.next())) != null && j < kvLimit) {
// Iterates through key/value pairs
++j;
for (Iterator<Cell> it : codecIterators) {

View File

@ -23,9 +23,11 @@ import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
@ -53,8 +55,8 @@ public class EncodedSeekPerformanceTest {
numberOfSeeks = DEFAULT_NUMBER_OF_SEEKS;
}
private List<KeyValue> prepareListOfTestSeeks(Path path) throws IOException {
List<KeyValue> allKeyValues = new ArrayList<KeyValue>();
private List<Cell> prepareListOfTestSeeks(Path path) throws IOException {
List<Cell> allKeyValues = new ArrayList<Cell>();
// read all of the key values
StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(),
@ -62,7 +64,7 @@ public class EncodedSeekPerformanceTest {
StoreFile.Reader reader = storeFile.createReader();
StoreFileScanner scanner = reader.getStoreFileScanner(true, false);
KeyValue current;
Cell current;
scanner.seek(KeyValue.LOWESTKEY);
while (null != (current = scanner.next())) {
@ -72,9 +74,9 @@ public class EncodedSeekPerformanceTest {
storeFile.closeReader(cacheConf.shouldEvictOnClose());
// pick seeks by random
List<KeyValue> seeks = new ArrayList<KeyValue>();
List<Cell> seeks = new ArrayList<Cell>();
for (int i = 0; i < numberOfSeeks; ++i) {
KeyValue keyValue = allKeyValues.get(
Cell keyValue = allKeyValues.get(
randomizer.nextInt(allKeyValues.size()));
seeks.add(keyValue);
}
@ -85,7 +87,7 @@ public class EncodedSeekPerformanceTest {
}
private void runTest(Path path, DataBlockEncoding blockEncoding,
List<KeyValue> seeks) throws IOException {
List<Cell> seeks) throws IOException {
// read all of the key values
StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE);
@ -96,25 +98,25 @@ public class EncodedSeekPerformanceTest {
StoreFileScanner scanner = reader.getStoreFileScanner(true, false);
long startReadingTime = System.nanoTime();
KeyValue current;
Cell current;
scanner.seek(KeyValue.LOWESTKEY);
while (null != (current = scanner.next())) { // just iterate it!
if (current.getLength() < 0) {
if (KeyValueUtil.ensureKeyValue(current).getLength() < 0) {
throw new IOException("Negative KV size: " + current);
}
totalSize += current.getLength();
totalSize += KeyValueUtil.ensureKeyValue(current).getLength();
}
long finishReadingTime = System.nanoTime();
// do seeks
long startSeeksTime = System.nanoTime();
for (KeyValue keyValue : seeks) {
for (Cell keyValue : seeks) {
scanner.seek(keyValue);
KeyValue toVerify = scanner.next();
Cell toVerify = scanner.next();
if (!keyValue.equals(toVerify)) {
System.out.println(String.format("KeyValue doesn't match:\n" +
"Orig key: %s\n" +
"Ret key: %s", keyValue.getKeyString(), toVerify.getKeyString()));
System.out.println(String.format("KeyValue doesn't match:\n" + "Orig key: %s\n"
+ "Ret key: %s", KeyValueUtil.ensureKeyValue(keyValue).getKeyString(), KeyValueUtil
.ensureKeyValue(toVerify).getKeyString()));
break;
}
}
@ -146,7 +148,7 @@ public class EncodedSeekPerformanceTest {
*/
public void runTests(Path path, DataBlockEncoding[] encodings)
throws IOException {
List<KeyValue> seeks = prepareListOfTestSeeks(path);
List<Cell> seeks = prepareListOfTestSeeks(path);
for (DataBlockEncoding blockEncoding : encodings) {
runTest(path, blockEncoding, seeks);

View File

@ -574,7 +574,7 @@ public class HFileReadWriteTest {
return false;
}
for (int i = 0; i < rand.nextInt(10) + 1; ++i) {
KeyValue kv = scanner.next();
Cell kv = scanner.next();
numKV.incrementAndGet();
if (i == 0 && kv == null) {
error("scanner.next() returned null at the first iteration for " +
@ -584,9 +584,10 @@ public class HFileReadWriteTest {
if (kv == null)
break;
String keyHashStr = MD5Hash.getMD5AsHex(kv.getKey());
KeyValue keyv = KeyValueUtil.ensureKeyValue(kv);
String keyHashStr = MD5Hash.getMD5AsHex(keyv.getKey());
keysRead.add(keyHashStr);
totalBytes.addAndGet(kv.getLength());
totalBytes.addAndGet(keyv.getLength());
}
}

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
@ -204,7 +205,7 @@ public class TestDefaultMemStore extends TestCase {
List<KeyValueScanner> memstorescanners = this.memstore.getScanners(mvcc.memstoreReadPoint());
assertEquals(1, memstorescanners.size());
final KeyValueScanner scanner = memstorescanners.get(0);
scanner.seek(KeyValue.createFirstOnRow(HConstants.EMPTY_START_ROW));
scanner.seek(KeyValueUtil.createFirstOnRow(HConstants.EMPTY_START_ROW));
assertEquals(kv1, scanner.next());
assertEquals(kv2, scanner.next());
assertNull(scanner.next());
@ -212,11 +213,11 @@ public class TestDefaultMemStore extends TestCase {
private void assertScannerResults(KeyValueScanner scanner, KeyValue[] expected)
throws IOException {
scanner.seek(KeyValue.createFirstOnRow(new byte[]{}));
scanner.seek(KeyValueUtil.createFirstOnRow(new byte[]{}));
List<Cell> returned = Lists.newArrayList();
while (true) {
KeyValue next = scanner.next();
Cell next = scanner.next();
if (next == null) break;
returned.add(next);
}
@ -417,7 +418,7 @@ public class TestDefaultMemStore extends TestCase {
KeyValueScanner s = this.memstore.getScanners(mvcc.memstoreReadPoint()).get(0);
s.seek(kv);
KeyValue ret = s.next();
Cell ret = s.next();
assertNotNull("Didnt find own write at all", ret);
assertEquals("Didnt read own writes",
kv.getTimestamp(), ret.getTimestamp());
@ -1009,7 +1010,7 @@ public class TestDefaultMemStore extends TestCase {
static void doScan(MemStore ms, int iteration) throws IOException {
long nanos = System.nanoTime();
KeyValueScanner s = ms.getScanners(0).get(0);
s.seek(KeyValue.createFirstOnRow(new byte[]{}));
s.seek(KeyValueUtil.createFirstOnRow(new byte[]{}));
System.out.println(iteration + " create/seek took: " + (System.nanoTime() - nanos)/1000);
int cnt=0;

View File

@ -21,11 +21,13 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CollectionBackedScanner;
import org.junit.experimental.categories.Category;
@ -103,7 +105,7 @@ public class TestKeyValueHeap extends HBaseTestCase {
KeyValueHeap kvh =
new KeyValueHeap(scanners, KeyValue.COMPARATOR);
List<KeyValue> actual = new ArrayList<KeyValue>();
List<Cell> actual = new ArrayList<Cell>();
while(kvh.peek() != null){
actual.add(kvh.next());
}
@ -159,7 +161,7 @@ public class TestKeyValueHeap extends HBaseTestCase {
KeyValue seekKv = new KeyValue(row2, fam1, null, null);
kvh.seek(seekKv);
List<KeyValue> actual = new ArrayList<KeyValue>();
List<Cell> actual = new ArrayList<Cell>();
actual.add(kvh.peek());
assertEquals(expected.size(), actual.size());
@ -209,7 +211,7 @@ public class TestKeyValueHeap extends HBaseTestCase {
}
private static class Scanner extends CollectionBackedScanner {
private Iterator<KeyValue> iter;
private Iterator<Cell> iter;
private KeyValue current;
private boolean closed = false;

View File

@ -22,7 +22,12 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.experimental.categories.Category;
@ -42,19 +47,19 @@ public class TestKeyValueScanFixture extends TestCase {
KeyValueScanner scan = new KeyValueScanFixture(
KeyValue.COMPARATOR, kvs);
KeyValue kv = KeyValue.createFirstOnRow(Bytes.toBytes("RowA"));
KeyValue kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowA"));
// should seek to this:
assertTrue(scan.seek(kv));
KeyValue res = scan.peek();
Cell res = scan.peek();
assertEquals(kvs[0], res);
kv = KeyValue.createFirstOnRow(Bytes.toBytes("RowB"));
kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowB"));
assertTrue(scan.seek(kv));
res = scan.peek();
assertEquals(kvs[2], res);
// ensure we pull things out properly:
kv = KeyValue.createFirstOnRow(Bytes.toBytes("RowA"));
kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowA"));
assertTrue(scan.seek(kv));
assertEquals(kvs[0], scan.peek());
assertEquals(kvs[0], scan.next());

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
@ -176,7 +177,7 @@ public class TestReversibleScanners {
for (int i = startRowNum; i >= 0; i--) {
if (i - 2 < 0) break;
i = i - 2;
kvHeap.seekToPreviousRow(KeyValue.createFirstOnRow(ROWS[i + 1]));
kvHeap.seekToPreviousRow(KeyValueUtil.createFirstOnRow(ROWS[i + 1]));
Pair<Integer, Integer> nextReadableNum = getNextReadableNumWithBackwardScan(
i, 0, readPoint);
if (nextReadableNum == null) break;
@ -457,7 +458,7 @@ public class TestReversibleScanners {
for (int i = startRowNum; i >= 0; i--) {
if (i % 2 == 1 && i - 2 >= 0) {
i = i - 2;
kvHeap.seekToPreviousRow(KeyValue.createFirstOnRow(ROWS[i + 1]));
kvHeap.seekToPreviousRow(KeyValueUtil.createFirstOnRow(ROWS[i + 1]));
}
for (int j = 0; j < QUALSIZE; j++) {
if (j % 2 == 1 && (j + 1) < QUALSIZE) {
@ -499,7 +500,7 @@ public class TestReversibleScanners {
scanner.seekToLastRow();
}
} else {
KeyValue startKey = KeyValue.createFirstOnRow(startRow);
KeyValue startKey = KeyValueUtil.createFirstOnRow(startRow);
for (KeyValueScanner scanner : scanners) {
scanner.backwardSeek(startKey);
}
@ -525,17 +526,17 @@ public class TestReversibleScanners {
// Case2: seek to the previous row in backwardSeek
int seekRowNum = ROWSIZE - 2;
assertTrue(scanner.backwardSeek(KeyValue.createLastOnRow(ROWS[seekRowNum])));
assertTrue(scanner.backwardSeek(KeyValueUtil.createLastOnRow(ROWS[seekRowNum])));
KeyValue expectedKey = makeKV(seekRowNum - 1, 0);
assertEquals(expectedKey, scanner.peek());
// Case3: unable to backward seek
assertFalse(scanner.backwardSeek(KeyValue.createLastOnRow(ROWS[0])));
assertFalse(scanner.backwardSeek(KeyValueUtil.createLastOnRow(ROWS[0])));
assertEquals(null, scanner.peek());
// Test seek to previous row
seekRowNum = ROWSIZE - 4;
assertTrue(scanner.seekToPreviousRow(KeyValue
assertTrue(scanner.seekToPreviousRow(KeyValueUtil
.createFirstOnRow(ROWS[seekRowNum])));
expectedKey = makeKV(seekRowNum - 1, 0);
assertEquals(expectedKey, scanner.peek());
@ -566,7 +567,7 @@ public class TestReversibleScanners {
// Case2: seek to the previous row in backwardSeek
int seekRowNum = ROWSIZE - 3;
KeyValue seekKey = KeyValue.createLastOnRow(ROWS[seekRowNum]);
KeyValue seekKey = KeyValueUtil.createLastOnRow(ROWS[seekRowNum]);
expectedKey = getNextReadableKeyValueWithBackwardScan(seekRowNum - 1, 0,
readPoint);
assertEquals(expectedKey != null, scanner.backwardSeek(seekKey));
@ -576,7 +577,7 @@ public class TestReversibleScanners {
seekRowNum = ROWSIZE - 4;
expectedKey = getNextReadableKeyValueWithBackwardScan(seekRowNum - 1, 0,
readPoint);
assertEquals(expectedKey != null, scanner.seekToPreviousRow(KeyValue
assertEquals(expectedKey != null, scanner.seekToPreviousRow(KeyValueUtil
.createFirstOnRow(ROWS[seekRowNum])));
assertEquals(expectedKey, scanner.peek());
}

View File

@ -33,11 +33,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Scan;
@ -590,7 +592,7 @@ public class TestStoreFile extends HBaseTestCase {
// Now do reseek with empty KV to position to the beginning of the file
KeyValue k = KeyValue.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY);
KeyValue k = KeyValueUtil.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY);
StoreFileScanner s = reader.getStoreFileScanner(false, false);
s.reseek(k);
@ -882,14 +884,16 @@ public class TestStoreFile extends HBaseTestCase {
readerTwo.loadFileInfo();
StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true);
scannerTwo.seek(KeyValue.LOWESTKEY);
KeyValue kv1 = null;
KeyValue kv2 = null;
Cell kv1 = null;
Cell kv2 = null;
while ((kv1 = scannerOne.next()) != null) {
kv2 = scannerTwo.next();
assertTrue(kv1.equals(kv2));
KeyValue keyv1 = KeyValueUtil.ensureKeyValue(kv1);
KeyValue keyv2 = KeyValueUtil.ensureKeyValue(kv2);
assertTrue(Bytes.compareTo(
kv1.getBuffer(), kv1.getKeyOffset(), kv1.getKeyLength(),
kv2.getBuffer(), kv2.getKeyOffset(), kv2.getKeyLength()) == 0);
keyv1.getBuffer(), keyv1.getKeyOffset(), keyv1.getKeyLength(),
keyv2.getBuffer(), keyv2.getKeyOffset(), keyv2.getKeyLength()) == 0);
assertTrue(Bytes.compareTo(
kv1.getValueArray(), kv1.getValueOffset(), kv1.getValueLength(),
kv2.getValueArray(), kv2.getValueOffset(), kv2.getValueLength()) == 0);

View File

@ -27,8 +27,10 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -73,15 +75,15 @@ public class TestStoreFileScannerWithTagCompression {
StoreFileScanner s = reader.getStoreFileScanner(false, false);
try {
// Now do reseek with empty KV to position to the beginning of the file
KeyValue k = KeyValue.createFirstOnRow(Bytes.toBytes("k2"));
KeyValue k = KeyValueUtil.createFirstOnRow(Bytes.toBytes("k2"));
s.reseek(k);
KeyValue kv = s.next();
Cell kv = s.next();
kv = s.next();
kv = s.next();
byte[] key5 = Bytes.toBytes("k5");
assertTrue(Bytes.equals(key5, 0, key5.length, kv.getRowArray(), kv.getRowOffset(),
kv.getRowLength()));
List<Tag> tags = kv.getTags();
List<Tag> tags = KeyValueUtil.ensureKeyValue(kv).getTags();
assertEquals(1, tags.size());
assertEquals("tag3", Bytes.toString(tags.get(0).getValue()));
} finally {