HBASE-9334 Convert KeyValue to Cell in hbase-client module - Filters

HBASE-9359 Convert KeyValue to Cell in hbase-client module - Result/Put/Delete, ColumnInterpreter


git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1519077 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-08-30 20:31:47 +00:00
parent 3ccdefbd57
commit 5a3060ee37
154 changed files with 2017 additions and 1457 deletions

View File

@ -15,11 +15,11 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
/**
* Coprocess interface.
*/
@ -53,4 +53,5 @@ public interface Coprocessor {
void start(CoprocessorEnvironment env) throws IOException;
void stop(CoprocessorEnvironment env) throws IOException;
}

View File

@ -26,6 +26,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -375,8 +377,9 @@ public class ClientScanner extends AbstractClientScanner {
if (values != null && values.length > 0) {
for (Result rs : values) {
cache.add(rs);
for (KeyValue kv : rs.raw()) {
remainingResultSize -= kv.heapSize();
for (Cell kv : rs.raw()) {
// TODO make method in Cell or CellUtil
remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
}
countdown--;
this.lastResult = rs;

View File

@ -27,6 +27,7 @@ import java.util.Map;
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.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
@ -149,18 +150,18 @@ public class Delete extends Mutation implements Comparable<Row> {
* @throws IOException
*/
@SuppressWarnings("unchecked")
public Delete addDeleteMarker(KeyValue kv) throws IOException {
public Delete addDeleteMarker(Cell kv) throws IOException {
// TODO: Deprecate and rename 'add' so it matches how we add KVs to Puts.
if (!kv.isDelete()) {
if (!CellUtil.isDelete(kv)) {
throw new IOException("The recently added KeyValue is not of type "
+ "delete. Rowkey: " + Bytes.toStringBinary(this.row));
}
if (Bytes.compareTo(this.row, 0, row.length, kv.getBuffer(),
if (Bytes.compareTo(this.row, 0, row.length, kv.getRowArray(),
kv.getRowOffset(), kv.getRowLength()) != 0) {
throw new WrongRowIOException("The row in " + kv.toString() +
" doesn't match the original one " + Bytes.toStringBinary(this.row));
}
byte [] family = kv.getFamily();
byte [] family = CellUtil.getFamilyArray(kv);
List<Cell> list = familyMap.get(family);
if (list == null) {
list = new ArrayList<Cell>();

View File

@ -195,7 +195,7 @@ public class Get extends OperationWithAttributes
/**
* Apply the specified server-side filter when performing the Get.
* Only {@link Filter#filterKeyValue(KeyValue)} is called AFTER all tests
* Only {@link Filter#filterKeyValue(Cell)} is called AFTER all tests
* for ttl, column match, deletes and max versions have been run.
* @param filter filter to run on the server
* @return this for invocation chaining

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@ -29,9 +28,9 @@ import java.util.TreeMap;
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.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
@ -122,7 +121,6 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
* @param value column value
* @return this
*/
@SuppressWarnings("unchecked")
public Put add(byte [] family, byte [] qualifier, long ts, byte [] value) {
if (ts < 0) {
throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + ts);
@ -130,7 +128,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
List<Cell> list = getCellList(family);
KeyValue kv = createPutKeyValue(family, qualifier, ts, value);
list.add(kv);
familyMap.put(kv.getFamily(), list);
familyMap.put(CellUtil.getFamilyArray(kv), list);
return this;
}
@ -142,13 +140,12 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
* @return this
* @throws java.io.IOException e
*/
@SuppressWarnings("unchecked")
public Put add(KeyValue kv) throws IOException{
byte [] family = kv.getFamily();
public Put add(Cell kv) throws IOException{
byte [] family = CellUtil.getFamilyArray(kv);
List<Cell> list = getCellList(family);
//Checking that the row of the kv is the same as the put
int res = Bytes.compareTo(this.row, 0, row.length,
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength());
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength());
if (res != 0) {
throw new WrongRowIOException("The row in " + kv.toString() +
" doesn't match the original one " + Bytes.toStringBinary(this.row));
@ -245,35 +242,31 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
// F F => 1
if (!ignoreTS && !ignoreValue) {
for (Cell cell : list) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (Arrays.equals(kv.getFamily(), family) &&
Arrays.equals(kv.getQualifier(), qualifier) &&
Arrays.equals(kv.getValue(), value) &&
kv.getTimestamp() == ts) {
if (CellUtil.matchingFamily(cell, family) &&
CellUtil.matchingQualifier(cell, qualifier) &&
CellUtil.matchingValue(cell, value) &&
cell.getTimestamp() == ts) {
return true;
}
}
} else if (ignoreValue && !ignoreTS) {
for (Cell cell : list) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (Arrays.equals(kv.getFamily(), family) && Arrays.equals(kv.getQualifier(), qualifier)
&& kv.getTimestamp() == ts) {
if (CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
&& cell.getTimestamp() == ts) {
return true;
}
}
} else if (!ignoreValue && ignoreTS) {
for (Cell cell : list) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (Arrays.equals(kv.getFamily(), family) && Arrays.equals(kv.getQualifier(), qualifier)
&& Arrays.equals(kv.getValue(), value)) {
if (CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
&& CellUtil.matchingValue(cell, value)) {
return true;
}
}
} else {
for (Cell cell : list) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (Arrays.equals(kv.getFamily(), family) &&
Arrays.equals(kv.getQualifier(), qualifier)) {
if (CellUtil.matchingFamily(cell, family) &&
CellUtil.matchingQualifier(cell, qualifier)) {
return true;
}
}
@ -287,14 +280,13 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
* @param family column family
* @param qualifier column qualifier
* @return a list of KeyValue objects with the matching family and qualifier,
* returns an empty list if one doesnt exist for the given family.
* returns an empty list if one doesn't exist for the given family.
*/
public List<KeyValue> get(byte[] family, byte[] qualifier) {
List<KeyValue> filteredList = new ArrayList<KeyValue>();
public List<Cell> get(byte[] family, byte[] qualifier) {
List<Cell> filteredList = new ArrayList<Cell>();
for (Cell cell: getCellList(family)) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (Arrays.equals(kv.getQualifier(), qualifier)) {
filteredList.add(kv);
if (CellUtil.matchingQualifier(cell, qualifier)) {
filteredList.add(cell);
}
}
return filteredList;

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -71,7 +72,7 @@ import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Public
@InterfaceStability.Stable
public class Result implements CellScannable {
private KeyValue [] kvs;
private Cell[] cells;
// We're not using java serialization. Transient here is just a marker to say
// that this is where we cache row if we're ever asked for it.
private transient byte [] row = null;
@ -97,10 +98,10 @@ public class Result implements CellScannable {
* Instantiate a Result with the specified array of KeyValues.
* <br><strong>Note:</strong> You must ensure that the keyvalues
* are already sorted
* @param kvs array of KeyValues
* @param cells array of KeyValues
*/
public Result(KeyValue [] kvs) {
this.kvs = kvs;
public Result(Cell [] cells) {
this.cells = cells;
}
/**
@ -109,9 +110,9 @@ public class Result implements CellScannable {
* are already sorted
* @param kvs List of KeyValues
*/
public Result(List<? extends Cell> kvs) {
public Result(List<Cell> kvs) {
// TODO: Here we presume the passed in Cells are KVs. One day this won't always be so.
this(kvs.toArray(new KeyValue[kvs.size()]));
this(kvs.toArray(new Cell[kvs.size()]));
}
/**
@ -121,65 +122,65 @@ public class Result implements CellScannable {
*/
public byte [] getRow() {
if (this.row == null) {
this.row = this.kvs == null || this.kvs.length == 0? null: this.kvs[0].getRow();
this.row = this.cells == null || this.cells.length == 0? null: CellUtil.getRowArray(this.cells[0]);
}
return this.row;
}
/**
* Return the array of KeyValues backing this Result instance.
* Return the array of Cells backing this Result instance.
*
* The array is sorted from smallest -> largest using the
* {@link KeyValue#COMPARATOR}.
*
* The array only contains what your Get or Scan specifies and no more.
* For example if you request column "A" 1 version you will have at most 1
* KeyValue in the array. If you request column "A" with 2 version you will
* have at most 2 KeyValues, with the first one being the newer timestamp and
* Cell in the array. If you request column "A" with 2 version you will
* have at most 2 Cells, with the first one being the newer timestamp and
* the second being the older timestamp (this is the sort order defined by
* {@link KeyValue#COMPARATOR}). If columns don't exist, they won't be
* present in the result. Therefore if you ask for 1 version all columns,
* it is safe to iterate over this array and expect to see 1 KeyValue for
* it is safe to iterate over this array and expect to see 1 Cell for
* each column and no more.
*
* This API is faster than using getFamilyMap() and getMap()
*
* @return array of KeyValues; can be null if nothing in the result
* @return array of Cells; can be null if nothing in the result
*/
public KeyValue[] raw() {
return kvs;
public Cell[] raw() {
return cells;
}
/**
* Create a sorted list of the KeyValue's in this result.
* Create a sorted list of the Cell's in this result.
*
* Since HBase 0.20.5 this is equivalent to raw().
*
* @return The sorted list of KeyValue's.
* @return The sorted list of Cell's.
*/
public List<KeyValue> list() {
public List<Cell> list() {
return isEmpty()? null: Arrays.asList(raw());
}
/**
* Return the KeyValues for the specific column. The KeyValues are sorted in
* Return the Cells for the specific column. The Cells are sorted in
* the {@link KeyValue#COMPARATOR} order. That implies the first entry in
* the list is the most recent column. If the query (Scan or Get) only
* requested 1 version the list will contain at most 1 entry. If the column
* did not exist in the result set (either the column does not exist
* or the column was not selected in the query) the list will be empty.
*
* Also see getColumnLatest which returns just a KeyValue
* Also see getColumnLatest which returns just a Cell
*
* @param family the family
* @param qualifier
* @return a list of KeyValues for this column or empty list if the column
* @return a list of Cells for this column or empty list if the column
* did not exist in the result set
*/
public List<KeyValue> getColumn(byte [] family, byte [] qualifier) {
List<KeyValue> result = new ArrayList<KeyValue>();
public List<Cell> getColumn(byte [] family, byte [] qualifier) {
List<Cell> result = new ArrayList<Cell>();
KeyValue [] kvs = raw();
Cell [] kvs = raw();
if (kvs == null || kvs.length == 0) {
return result;
@ -190,7 +191,7 @@ public class Result implements CellScannable {
}
for (int i = pos ; i < kvs.length ; i++ ) {
KeyValue kv = kvs[i];
KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[i]);
if (kv.matchingColumn(family,qualifier)) {
result.add(kv);
} else {
@ -201,11 +202,11 @@ public class Result implements CellScannable {
return result;
}
protected int binarySearch(final KeyValue [] kvs,
protected int binarySearch(final Cell [] kvs,
final byte [] family,
final byte [] qualifier) {
KeyValue searchTerm =
KeyValue.createFirstOnRow(kvs[0].getRow(),
Cell searchTerm =
KeyValue.createFirstOnRow(CellUtil.getRowArray(kvs[0]),
family, qualifier);
// pos === ( -(insertion point) - 1)
@ -234,7 +235,7 @@ public class Result implements CellScannable {
*
* @return the index where the value was found, or -1 otherwise
*/
protected int binarySearch(final KeyValue [] kvs,
protected int binarySearch(final Cell [] kvs,
final byte [] family, final int foffset, final int flength,
final byte [] qualifier, final int qoffset, final int qlength) {
@ -246,8 +247,8 @@ public class Result implements CellScannable {
buffer = new byte[(int) Math.ceil(keyValueSize / PAD_WIDTH) * PAD_WIDTH];
}
KeyValue searchTerm = KeyValue.createFirstOnRow(buffer, 0,
kvs[0].getBuffer(), kvs[0].getRowOffset(), kvs[0].getRowLength(),
Cell searchTerm = KeyValue.createFirstOnRow(buffer, 0,
kvs[0].getRowArray(), kvs[0].getRowOffset(), kvs[0].getRowLength(),
family, foffset, flength,
qualifier, qoffset, qlength);
@ -265,16 +266,16 @@ public class Result implements CellScannable {
}
/**
* The KeyValue for the most recent timestamp for a given column.
* The Cell for the most recent timestamp for a given column.
*
* @param family
* @param qualifier
*
* @return the KeyValue for the column, or null if no value exists in the row or none have been
* @return the Cell for the column, or null if no value exists in the row or none have been
* selected in the query (Get/Scan)
*/
public KeyValue getColumnLatest(byte [] family, byte [] qualifier) {
KeyValue [] kvs = raw(); // side effect possibly.
public Cell getColumnLatest(byte [] family, byte [] qualifier) {
Cell [] kvs = raw(); // side effect possibly.
if (kvs == null || kvs.length == 0) {
return null;
}
@ -282,7 +283,7 @@ public class Result implements CellScannable {
if (pos == -1) {
return null;
}
KeyValue kv = kvs[pos];
KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[pos]);
if (kv.matchingColumn(family, qualifier)) {
return kv;
}
@ -290,7 +291,7 @@ public class Result implements CellScannable {
}
/**
* The KeyValue for the most recent timestamp for a given column.
* The Cell for the most recent timestamp for a given column.
*
* @param family family name
* @param foffset family offset
@ -299,13 +300,13 @@ public class Result implements CellScannable {
* @param qoffset qualifier offset
* @param qlength qualifier length
*
* @return the KeyValue for the column, or null if no value exists in the row or none have been
* @return the Cell for the column, or null if no value exists in the row or none have been
* selected in the query (Get/Scan)
*/
public KeyValue getColumnLatest(byte [] family, int foffset, int flength,
public Cell getColumnLatest(byte [] family, int foffset, int flength,
byte [] qualifier, int qoffset, int qlength) {
KeyValue [] kvs = raw(); // side effect possibly.
Cell [] kvs = raw(); // side effect possibly.
if (kvs == null || kvs.length == 0) {
return null;
}
@ -313,7 +314,7 @@ public class Result implements CellScannable {
if (pos == -1) {
return null;
}
KeyValue kv = kvs[pos];
KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[pos]);
if (kv.matchingColumn(family, foffset, flength, qualifier, qoffset, qlength)) {
return kv;
}
@ -327,11 +328,11 @@ public class Result implements CellScannable {
* @return value of latest version of column, null if none found
*/
public byte[] getValue(byte [] family, byte [] qualifier) {
KeyValue kv = getColumnLatest(family, qualifier);
Cell kv = getColumnLatest(family, qualifier);
if (kv == null) {
return null;
}
return kv.getValue();
return CellUtil.getValueArray(kv);
}
/**
@ -344,12 +345,12 @@ public class Result implements CellScannable {
*/
public ByteBuffer getValueAsByteBuffer(byte [] family, byte [] qualifier) {
KeyValue kv = getColumnLatest(family, 0, family.length, qualifier, 0, qualifier.length);
Cell kv = getColumnLatest(family, 0, family.length, qualifier, 0, qualifier.length);
if (kv == null) {
return null;
}
return kv.getValueAsByteBuffer();
return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
}
/**
@ -367,12 +368,12 @@ public class Result implements CellScannable {
public ByteBuffer getValueAsByteBuffer(byte [] family, int foffset, int flength,
byte [] qualifier, int qoffset, int qlength) {
KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
if (kv == null) {
return null;
}
return kv.getValueAsByteBuffer();
return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
}
/**
@ -413,12 +414,12 @@ public class Result implements CellScannable {
public boolean loadValue(byte [] family, int foffset, int flength,
byte [] qualifier, int qoffset, int qlength, ByteBuffer dst)
throws BufferOverflowException {
KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
if (kv == null) {
return false;
}
kv.loadValue(dst);
dst.put(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
return true;
}
@ -450,7 +451,7 @@ public class Result implements CellScannable {
public boolean containsNonEmptyColumn(byte [] family, int foffset, int flength,
byte [] qualifier, int qoffset, int qlength) {
KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
return (kv != null) && (kv.getValueLength() > 0);
}
@ -482,7 +483,7 @@ public class Result implements CellScannable {
*/
public boolean containsEmptyColumn(byte [] family, int foffset, int flength,
byte [] qualifier, int qoffset, int qlength) {
KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
return (kv != null) && (kv.getValueLength() == 0);
}
@ -496,7 +497,7 @@ public class Result implements CellScannable {
* @return true if at least one value exists in the result, false if not
*/
public boolean containsColumn(byte [] family, byte [] qualifier) {
KeyValue kv = getColumnLatest(family, qualifier);
Cell kv = getColumnLatest(family, qualifier);
return kv != null;
}
@ -535,8 +536,8 @@ public class Result implements CellScannable {
return null;
}
this.familyMap = new TreeMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>(Bytes.BYTES_COMPARATOR);
for(KeyValue kv : this.kvs) {
byte [] family = kv.getFamily();
for(Cell kv : this.cells) {
byte [] family = CellUtil.getFamilyArray(kv);
NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap =
familyMap.get(family);
if(columnMap == null) {
@ -544,7 +545,7 @@ public class Result implements CellScannable {
(Bytes.BYTES_COMPARATOR);
familyMap.put(family, columnMap);
}
byte [] qualifier = kv.getQualifier();
byte [] qualifier = CellUtil.getQualifierArray(kv);
NavigableMap<Long, byte[]> versionMap = columnMap.get(qualifier);
if(versionMap == null) {
versionMap = new TreeMap<Long, byte[]>(new Comparator<Long>() {
@ -555,7 +556,7 @@ public class Result implements CellScannable {
columnMap.put(qualifier, versionMap);
}
Long timestamp = kv.getTimestamp();
byte [] value = kv.getValue();
byte [] value = CellUtil.getValueArray(kv);
versionMap.put(timestamp, value);
}
@ -632,22 +633,22 @@ public class Result implements CellScannable {
if (isEmpty()) {
return null;
}
return kvs[0].getValue();
return CellUtil.getValueArray(cells[0]);
}
/**
* Check if the underlying KeyValue [] is empty or not
* Check if the underlying Cell [] is empty or not
* @return true if empty
*/
public boolean isEmpty() {
return this.kvs == null || this.kvs.length == 0;
return this.cells == null || this.cells.length == 0;
}
/**
* @return the size of the underlying KeyValue []
* @return the size of the underlying Cell []
*/
public int size() {
return this.kvs == null? 0: this.kvs.length;
return this.cells == null? 0: this.cells.length;
}
/**
@ -663,7 +664,7 @@ public class Result implements CellScannable {
}
sb.append("{");
boolean moreThanOne = false;
for(KeyValue kv : this.kvs) {
for(Cell kv : this.cells) {
if(moreThanOne) {
sb.append(", ");
} else {
@ -691,11 +692,11 @@ public class Result implements CellScannable {
throw new Exception("This row doesn't have the same number of KVs: "
+ res1.toString() + " compared to " + res2.toString());
}
KeyValue[] ourKVs = res1.raw();
KeyValue[] replicatedKVs = res2.raw();
Cell[] ourKVs = res1.raw();
Cell[] replicatedKVs = res2.raw();
for (int i = 0; i < res1.size(); i++) {
if (!ourKVs[i].equals(replicatedKVs[i]) ||
!Bytes.equals(ourKVs[i].getValue(), replicatedKVs[i].getValue())) {
!Bytes.equals(CellUtil.getValueArray(ourKVs[i]), CellUtil.getValueArray(replicatedKVs[i]))) {
throw new Exception("This result was different: "
+ res1.toString() + " compared to " + res2.toString());
}
@ -709,11 +710,11 @@ public class Result implements CellScannable {
public void copyFrom(Result other) {
this.row = null;
this.familyMap = null;
this.kvs = other.kvs;
this.cells = other.cells;
}
@Override
public CellScanner cellScanner() {
return CellUtil.createCellScanner(this.kvs);
return CellUtil.createCellScanner(this.cells);
}
}

View File

@ -18,22 +18,23 @@
package org.apache.hadoop.hbase.client;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import java.io.IOException;
import java.net.UnknownHostException;
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.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@ -46,8 +47,8 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
import java.io.IOException;
import java.net.UnknownHostException;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/**
* Scanner operations such as create, next, etc.
@ -252,8 +253,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
}
long resultSize = 0;
for (Result rr : rrs) {
for (KeyValue kv : rr.raw()) {
resultSize += kv.getLength();
for (Cell kv : rr.raw()) {
// TODO add getLength to Cell/use CellUtil#estimatedSizeOf
resultSize += KeyValueUtil.ensureKeyValue(kv).getLength();
}
}
this.scanMetrics.countOfBytesInResults.addAndGet(resultSize);

View File

@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
@ -792,7 +793,7 @@ public class AggregationClient {
for (int i = 0; i < results.length; i++) {
Result r = results[i];
// retrieve weight
KeyValue kv = r.getColumnLatest(colFamily, weightQualifier);
Cell kv = r.getColumnLatest(colFamily, weightQualifier);
R newValue = ci.getValue(colFamily, weightQualifier, kv);
S s = ci.castToReturnType(newValue);
double newSumVal = movingSumVal + ci.divideForAvg(s, 1L);

View File

@ -24,7 +24,8 @@ import java.math.RoundingMode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BigDecimalMsg;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
@ -43,12 +44,12 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
EmptyMsg, BigDecimalMsg, BigDecimalMsg> {
@Override
public BigDecimal getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
public BigDecimal getValue(byte[] colFamily, byte[] colQualifier, Cell kv)
throws IOException {
if (kv == null || kv.getValue() == null) {
if (kv == null || CellUtil.getValueArray(kv) == null) {
return null;
}
return Bytes.toBigDecimal(kv.getValue()).setScale(2, RoundingMode.HALF_EVEN);
return Bytes.toBigDecimal(CellUtil.getValueArray(kv)).setScale(2, RoundingMode.HALF_EVEN);
}
@Override

View File

@ -18,16 +18,16 @@
*/
package org.apache.hadoop.hbase.client.coprocessor;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LongMsg;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
/**
* a concrete column interpreter implementation. The cell value is a Long value
* and its promoted data type is also a Long value. For computing aggregation
@ -41,11 +41,11 @@ import java.io.IOException;
public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
EmptyMsg, LongMsg, LongMsg> {
public Long getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
public Long getValue(byte[] colFamily, byte[] colQualifier, Cell kv)
throws IOException {
if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG)
return null;
return Bytes.toLong(kv.getBuffer(), kv.getValueOffset());
return Bytes.toLong(kv.getValueArray(), kv.getValueOffset());
}
@Override

View File

@ -19,13 +19,16 @@
package org.apache.hadoop.hbase.coprocessor;
import com.google.protobuf.Message;
import java.io.IOException;
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.client.coprocessor.LongColumnInterpreter;
import java.io.IOException;
import com.google.protobuf.Message;
/**
* Defines how value for specific column is interpreted and provides utility
@ -59,14 +62,32 @@ public abstract class ColumnInterpreter<T, S, P extends Message,
Q extends Message, R extends Message> {
/**
* TODO: when removing {@link #getValue(byte[], byte[], KeyValue)}, this method should be made abstract
*
* @param colFamily
* @param colQualifier
* @param kv
* @param c
* @return value of type T
* @throws IOException
*/
public abstract T getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
throws IOException;
public T getValue(byte[] colFamily, byte[] colQualifier, Cell c)
throws IOException {
// call the deprecated method for compatiblity.
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
return getValue(colFamily, colQualifier, kv);
}
/**
* This method used to be abstract, and is preserved for compatibility and easy of conversion
* from 0.94->0.96.
*
* Please override {@link #getValue(byte[], byte[], Cell)} instead.
*/
@Deprecated
public T getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
throws IOException {
return null;
}
/**
* @param l1

View File

@ -19,15 +19,16 @@
package org.apache.hadoop.hbase.filter;
import com.google.common.base.Preconditions;
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.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Simple filter that returns first N columns on row only.
@ -56,7 +57,7 @@ public class ColumnCountGetFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell v) {
this.count++;
return filterAllRemaining() ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL;
}

View File

@ -23,6 +23,7 @@ import com.google.protobuf.ByteString;
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;
@ -103,13 +104,13 @@ public class ColumnPaginationFilter extends FilterBase
}
@Override
public ReturnCode filterKeyValue(KeyValue v)
public ReturnCode filterKeyValue(Cell v)
{
if (columnOffset != null) {
if (count >= limit) {
return ReturnCode.NEXT_ROW;
}
byte[] buffer = v.getBuffer();
byte[] buffer = v.getQualifierArray();
if (buffer == null) {
return ReturnCode.SEEK_NEXT_USING_HINT;
}
@ -141,9 +142,10 @@ public class ColumnPaginationFilter extends FilterBase
}
}
public KeyValue getNextKeyHint(KeyValue kv) {
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), columnOffset, 0, columnOffset.length);
}

View File

@ -24,6 +24,7 @@ import com.google.protobuf.ByteString;
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;
@ -50,11 +51,11 @@ public class ColumnPrefixFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
if (this.prefix == null || kv.getBuffer() == null) {
public ReturnCode filterKeyValue(Cell kv) {
if (this.prefix == null || kv.getQualifierArray() == null) {
return ReturnCode.INCLUDE;
} else {
return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
}
}
@ -127,9 +128,10 @@ public class ColumnPrefixFilter extends FilterBase {
return Bytes.equals(this.getPrefix(), other.getPrefix());
}
public KeyValue getNextKeyHint(KeyValue kv) {
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), prefix, 0, prefix.length);
}

View File

@ -26,6 +26,7 @@ import com.google.protobuf.ByteString;
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;
@ -114,8 +115,9 @@ public class ColumnRangeFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
byte[] buffer = kv.getBuffer();
public ReturnCode filterKeyValue(Cell kv) {
// TODO have a column compare method in Cell
byte[] buffer = kv.getQualifierArray();
int qualifierOffset = kv.getQualifierOffset();
int qualifierLength = kv.getQualifierLength();
int cmpMin = 1;
@ -213,9 +215,9 @@ public class ColumnRangeFilter extends FilterBase {
}
@Override
public KeyValue getNextKeyHint(KeyValue kv) {
return KeyValue.createFirstOnRow(kv.getBuffer(), kv.getRowOffset(), kv
.getRowLength(), kv.getBuffer(), kv.getFamilyOffset(), kv
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(kv.getRowArray(), kv.getRowOffset(), kv
.getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv
.getFamilyLength(), this.minColumn, 0, len(this.minColumn));
}

View File

@ -120,7 +120,8 @@ public abstract class CompareFilter extends FilterBase {
}
}
public static ArrayList extractArguments(ArrayList<byte []> filterArguments) {
// returns an array of heterogeneous objects
public static ArrayList<Object> extractArguments(ArrayList<byte []> filterArguments) {
Preconditions.checkArgument(filterArguments.size() == 2,
"Expected 2 but got: %s", filterArguments.size());
CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(0));
@ -135,7 +136,7 @@ public abstract class CompareFilter extends FilterBase {
" can only be used with EQUAL and NOT_EQUAL");
}
}
ArrayList arguments = new ArrayList();
ArrayList<Object> arguments = new ArrayList<Object>();
arguments.add(compareOp);
arguments.add(comparator);
return arguments;

View File

@ -18,17 +18,6 @@
*/
package org.apache.hadoop.hbase.filter;
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
@ -36,6 +25,20 @@ import java.util.Iterator;
import java.util.List;
import java.util.Set;
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.ProtobufUtil;
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.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter for adding inter-column timestamp matching
* Only cells with a correspondingly timestamped entry in
@ -77,9 +80,9 @@ public class DependentColumnFilter extends CompareFilter {
/**
* Constructor for DependentColumn filter.
* Keyvalues where a keyvalue from target column
* with the same timestamp do not exist will be dropped.
*
* Cells where a Cell from target column
* with the same timestamp do not exist will be dropped.
*
* @param family name of target column family
* @param qualifier name of column qualifier
*/
@ -89,12 +92,12 @@ public class DependentColumnFilter extends CompareFilter {
/**
* Constructor for DependentColumn filter.
* Keyvalues where a keyvalue from target column
* with the same timestamp do not exist will be dropped.
*
* Cells where a Cell from target column
* with the same timestamp do not exist will be dropped.
*
* @param family name of dependent column family
* @param qualifier name of dependent qualifier
* @param dropDependentColumn whether the dependent columns keyvalues should be discarded
* @param dropDependentColumn whether the dependent columns Cells should be discarded
*/
public DependentColumnFilter(final byte [] family, final byte [] qualifier,
final boolean dropDependentColumn) {
@ -132,7 +135,9 @@ public class DependentColumnFilter extends CompareFilter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell c) {
// 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)) {
// include non-matches for the time being, they'll be discarded afterwards
@ -140,7 +145,7 @@ public class DependentColumnFilter extends CompareFilter {
}
// If it doesn't pass the op, skip it
if (comparator != null
&& doCompare(compareOp, comparator, v.getBuffer(), v.getValueOffset(),
&& doCompare(compareOp, comparator, v.getValueArray(), v.getValueOffset(),
v.getValueLength()))
return ReturnCode.SKIP;
@ -152,9 +157,9 @@ public class DependentColumnFilter extends CompareFilter {
}
@Override
public void filterRow(List<KeyValue> kvs) {
Iterator<KeyValue> it = kvs.iterator();
KeyValue kv;
public void filterRowCells(List<Cell> kvs) {
Iterator<? extends Cell> it = kvs.iterator();
Cell kv;
while(it.hasNext()) {
kv = it.next();
if(!stampSet.contains(kv.getTimestamp())) {

View File

@ -19,16 +19,17 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import java.util.ArrayList;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter based on the column family. It takes an
@ -59,10 +60,10 @@ public class FamilyFilter extends CompareFilter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell v) {
int familyLength = v.getFamilyLength();
if (familyLength > 0) {
if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
if (doCompare(this.compareOp, this.comparator, v.getFamilyArray(),
v.getFamilyOffset(), familyLength)) {
return ReturnCode.SKIP;
}
@ -71,7 +72,7 @@ public class FamilyFilter extends CompareFilter {
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
ArrayList<?> arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
return new FamilyFilter(compareOp, comparator);

View File

@ -24,6 +24,7 @@ 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.KeyValue;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -35,11 +36,11 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
* <li> {@link #reset()} : reset the filter state before filtering a new row. </li>
* <li> {@link #filterAllRemaining()}: true means row scan is over; false means keep going. </li>
* <li> {@link #filterRowKey(byte[],int,int)}: true means drop this row; false means include.</li>
* <li> {@link #filterKeyValue(KeyValue)}: decides whether to include or exclude this KeyValue.
* <li> {@link #filterKeyValue(Cell)}: decides whether to include or exclude this KeyValue.
* See {@link ReturnCode}. </li>
* <li> {@link #transform(KeyValue)}: if the KeyValue is included, let the filter transform the
* KeyValue. </li>
* <li> {@link #filterRow(List)}: allows direct modification of the final list to be submitted
* <li> {@link #filterRowCells(List)}: allows direct modification of the final list to be submitted
* <li> {@link #filterRow()}: last chance to drop entire row based on the sequence of
* filter calls. Eg: filter a row if it doesn't contain a specified column. </li>
* </ul>
@ -67,7 +68,7 @@ public abstract class Filter {
/**
* Filters a row based on the row key. If this returns true, the entire row will be excluded. If
* false, each KeyValue in the row will be passed to {@link #filterKeyValue(KeyValue)} below.
* false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below.
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
@ -103,16 +104,16 @@ public abstract class Filter {
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param v the KeyValue in question
* @param v the Cell in question
* @return code as described below
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
* @see Filter.ReturnCode
*/
abstract public ReturnCode filterKeyValue(final KeyValue v) throws IOException;
abstract public ReturnCode filterKeyValue(final Cell v) throws IOException;
/**
* Give the filter a chance to transform the passed KeyValue. If the KeyValue is changed a new
* KeyValue object must be returned.
* Give the filter a chance to transform the passed KeyValue. If the Cell is changed a new
* Cell object must be returned.
*
* @see org.apache.hadoop.hbase.KeyValue#shallowCopy()
* The transformed KeyValue is what is eventually returned to the client. Most filters will
@ -127,22 +128,30 @@ public abstract class Filter {
* @return the changed KeyValue
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public KeyValue transform(final KeyValue v) throws IOException;
abstract public Cell transformCell(final Cell v) throws IOException;
/**
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
* This is for transition from 0.94 -> 0.96
**/
@Deprecated // use Cell transformCell(final Cell)
abstract public KeyValue transform(final KeyValue currentKV) throws IOException;
/**
* Return codes for filterValue().
*/
public enum ReturnCode {
/**
* Include the KeyValue
* Include the Cell
*/
INCLUDE,
/**
* Include the KeyValue and seek to the next column skipping older versions.
* Include the Cell and seek to the next column skipping older versions.
*/
INCLUDE_AND_NEXT_COL,
/**
* Skip this KeyValue
* Skip this Cell
*/
SKIP,
/**
@ -161,14 +170,21 @@ public abstract class Filter {
}
/**
* Chance to alter the list of keyvalues to be submitted. Modifications to the list will carry on
* Chance to alter the list of Cells to be submitted. Modifications to the list will carry on
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param kvs the list of keyvalues to be filtered
* @param kvs the list of Cells to be filtered
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public void filterRowCells(List<Cell> kvs) throws IOException;
/**
* WARNING: please to not override this method. Instead override {@link #filterRowCells(List)}.
* This is for transition from 0.94 -> 0.96
**/
@Deprecated
abstract public void filterRow(List<KeyValue> kvs) throws IOException;
/**
@ -180,7 +196,7 @@ public abstract class Filter {
abstract public boolean hasFilterRow();
/**
* Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)} calls. The filter
* Last chance to veto row based on previous {@link #filterKeyValue(Cell)} calls. The filter
* needs to retain state then return a particular value for this call if they wish to exclude a
* row if a certain column is missing (for example).
*
@ -192,6 +208,9 @@ public abstract class Filter {
*/
abstract public boolean filterRow() throws IOException;
@Deprecated // use Cell GetNextKeyHint(final Cell)
abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException;
/**
* If the filter returns the match code SEEK_NEXT_USING_HINT, then it should also tell which is
* the next key it must seek to. After receiving the match code SEEK_NEXT_USING_HINT, the
@ -204,7 +223,7 @@ public abstract class Filter {
* seek to next.
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException;
abstract public Cell getNextCellHint(final Cell currentKV) throws IOException;
/**
* Check that given column family is essential for filter to check row. Most filters always return

View File

@ -5,7 +5,8 @@
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
@ -24,7 +25,9 @@ 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.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
/**
* Abstract base class to help you implement new Filters. Common "ignore" or NOOP type
@ -72,12 +75,12 @@ public abstract class FilterBase extends Filter {
/**
* Filters that dont filter by key value can inherit this implementation that
* includes all KeyValues.
* includes all Cells.
*
* @inheritDoc
*/
@Override
public ReturnCode filterKeyValue(KeyValue ignored) throws IOException {
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}
@ -87,22 +90,54 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public KeyValue transform(KeyValue v) throws IOException {
return v;
public Cell transformCell(Cell v) throws IOException {
// Old filters based off of this class will override KeyValue transform(KeyValue).
// Thus to maintain compatibility we need to call the old version.
return transform(KeyValueUtil.ensureKeyValue(v));
}
/**
* Filters that never filter by modifying the returned List of KeyValues can
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
*
* This is for transition from 0.94 -> 0.96
*/
@Override
@Deprecated
public KeyValue transform(KeyValue currentKV) throws IOException {
return currentKV;
}
/**
* Filters that never filter by modifying the returned List of Cells can
* inherit this implementation that does nothing.
*
* @inheritDoc
*/
@Override
public void filterRow(List<KeyValue> ignored) throws IOException {
public void filterRowCells(List<Cell> ignored) throws IOException {
// Old filters based off of this class will override KeyValue transform(KeyValue).
// Thus to maintain compatibility we need to call the old version.
List<KeyValue> kvs = new ArrayList<KeyValue>(ignored.size());
for (Cell c : ignored) {
kvs.add(KeyValueUtil.ensureKeyValue(c));
}
filterRow(kvs);
ignored.clear();
ignored.addAll(kvs);
}
/**
* Fitlers that never filter by modifying the returned List of KeyValues can
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
*
* This is for transition from 0.94 -> 0.96
*/
@Override
@Deprecated
public void filterRow(List<KeyValue> kvs) throws IOException {
}
/**
* Fitlers that never filter by modifying the returned List of Cells can
* inherit this implementation that does nothing.
*
* @inheritDoc
@ -114,7 +149,7 @@ public abstract class FilterBase extends Filter {
/**
* Filters that never filter by rows based on previously gathered state from
* {@link #filterKeyValue(KeyValue)} can inherit this implementation that
* {@link #filterKeyValue(Cell)} can inherit this implementation that
* never filters a row.
*
* @inheritDoc
@ -124,14 +159,25 @@ public abstract class FilterBase extends Filter {
return false;
}
/**
* This method is deprecated and you should override Cell getNextKeyHint(Cell) instead.
*/
@Override
@Deprecated
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
return null;
}
/**
* Filters that are not sure which key must be next seeked to, can inherit
* this implementation that, by default, returns a null KeyValue.
* this implementation that, by default, returns a null Cell.
*
* @inheritDoc
*/
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
return null;
public Cell getNextCellHint(Cell currentKV) throws IOException {
// Old filters based off of this class will override KeyValue getNextKeyHint(KeyValue).
// Thus to maintain compatibility we need to call the old version.
return getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV));
}
/**

View File

@ -25,7 +25,9 @@ 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.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@ -51,7 +53,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FilterList extends Filter {
final public class FilterList extends Filter {
/** set operator */
public static enum Operator {
/** !AND */
@ -65,17 +67,17 @@ public class FilterList extends Filter {
private List<Filter> filters = new ArrayList<Filter>();
private Filter seekHintFilter = null;
/** Reference KeyValue used by {@link #transform(KeyValue)} for validation purpose. */
private KeyValue referenceKV = null;
/** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */
private Cell referenceKV = null;
/**
* When filtering a given KeyValue in {@link #filterKeyValue(KeyValue)},
* this stores the transformed KeyValue to be returned by {@link #transform(KeyValue)}.
* When filtering a given Cell in {@link #filterKeyValue(Cell)},
* this stores the transformed Cell to be returned by {@link #transformCell(Cell)}.
*
* Individual filters transformation are applied only when the filter includes the KeyValue.
* Individual filters transformation are applied only when the filter includes the Cell.
* Transformations are composed in the order specified by {@link #filters}.
*/
private KeyValue transformedKV = null;
private Cell transformedKV = null;
/**
* Constructor that takes a set of {@link Filter}s. The default operator
@ -201,22 +203,36 @@ public class FilterList extends Filter {
return operator == Operator.MUST_PASS_ONE;
}
@Override
public Cell transformCell(Cell v) throws IOException {
return transform(KeyValueUtil.ensureKeyValue(v));
}
/**
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
*
* When removing this, its body should be placed in transformCell.
*
* This is for transition from 0.94 -> 0.96
*/
@Deprecated
@Override
public KeyValue transform(KeyValue v) throws IOException {
// transform() is expected to follow an inclusive filterKeyValue() immediately:
if (!v.equals(this.referenceKV)) {
throw new IllegalStateException(
"Reference KeyValue: " + this.referenceKV + " does not match: " + v);
"Reference Cell: " + this.referenceKV + " does not match: " + v);
}
return this.transformedKV;
return KeyValueUtil.ensureKeyValue(this.transformedKV);
}
@Override
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
public ReturnCode filterKeyValue(Cell v) throws IOException {
this.referenceKV = v;
// Accumulates successive transformation of every filter that includes the KeyValue:
KeyValue transformed = v;
// Accumulates successive transformation of every filter that includes the Cell:
Cell transformed = v;
ReturnCode rc = operator == Operator.MUST_PASS_ONE?
ReturnCode.SKIP: ReturnCode.INCLUDE;
@ -231,7 +247,7 @@ public class FilterList extends Filter {
case INCLUDE_AND_NEXT_COL:
rc = ReturnCode.INCLUDE_AND_NEXT_COL;
case INCLUDE:
transformed = filter.transform(transformed);
transformed = filter.transformCell(transformed);
continue;
case SEEK_NEXT_USING_HINT:
seekHintFilter = filter;
@ -249,11 +265,11 @@ public class FilterList extends Filter {
if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
rc = ReturnCode.INCLUDE;
}
transformed = filter.transform(transformed);
transformed = filter.transformCell(transformed);
break;
case INCLUDE_AND_NEXT_COL:
rc = ReturnCode.INCLUDE_AND_NEXT_COL;
transformed = filter.transform(transformed);
transformed = filter.transformCell(transformed);
// must continue here to evaluate all filters
break;
case NEXT_ROW:
@ -270,19 +286,56 @@ public class FilterList extends Filter {
}
}
// Save the transformed KeyValue for transform():
// Save the transformed Cell for transform():
this.transformedKV = transformed;
return rc;
}
/**
* Filters that never filter by modifying the returned List of Cells can
* inherit this implementation that does nothing.
*
* @inheritDoc
*/
@Override
public void filterRow(List<KeyValue> kvs) throws IOException {
for (Filter filter : filters) {
filter.filterRow(kvs);
public void filterRowCells(List<Cell> ignored) throws IOException {
// Old filters based off of this class will override KeyValue transform(KeyValue).
// Thus to maintain compatibility we need to call the old version.
List<KeyValue> kvs = new ArrayList<KeyValue>(ignored.size());
for (Cell c : ignored) {
kvs.add(KeyValueUtil.ensureKeyValue(c));
}
filterRow(kvs);
ignored.clear();
ignored.addAll(kvs);
}
/**
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
*
* This is for transition from 0.94 -> 0.96
*/
@Override
@Deprecated
public void filterRow(List<KeyValue> kvs) throws IOException {
// when removing this, this body should be in filterRowCells
// convert to List<Cell> and call the new interface (this will call 0.96-style
// #filterRowCells(List<Cell>) which may delegate to legacy #filterRow(List<KV>)
List<Cell> cells = new ArrayList<Cell>(kvs.size());
cells.addAll(kvs);
for (Filter filter : filters) {
filter.filterRowCells(cells);
}
// convert results into kvs
kvs.clear();
for (Cell c : cells) {
kvs.add(KeyValueUtil.ensureKeyValue(c));
}
}
@Override
public boolean hasFilterRow() {
for (Filter filter : filters) {
@ -364,16 +417,22 @@ public class FilterList extends Filter {
}
@Override
@Deprecated
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
KeyValue keyHint = null;
return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
}
@Override
public Cell getNextCellHint(Cell currentKV) throws IOException {
Cell keyHint = null;
if (operator == Operator.MUST_PASS_ALL) {
keyHint = seekHintFilter.getNextKeyHint(currentKV);
keyHint = seekHintFilter.getNextCellHint(currentKV);
return keyHint;
}
// If any condition can pass, we need to keep the min hint
for (Filter filter : filters) {
KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
Cell curKeyHint = filter.getNextCellHint(currentKV);
if (curKeyHint == null) {
// If we ever don't have a hint and this is must-pass-one, then no hint
return null;

View File

@ -20,14 +20,18 @@
package org.apache.hadoop.hbase.filter;
import java.io.IOException;
import java.util.ArrayList;
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.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.zookeeper.KeeperException.UnimplementedException;
import com.google.protobuf.InvalidProtocolBufferException;
@ -40,7 +44,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FilterWrapper extends Filter {
final public class FilterWrapper extends Filter {
Filter filter = null;
public FilterWrapper( Filter filter ) {
@ -97,9 +101,23 @@ public class FilterWrapper extends Filter {
return this.filter.filterRow();
}
/**
* This method is deprecated and you should override Cell getNextKeyHint(Cell) instead.
*/
@Override
@Deprecated
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
return this.filter.getNextKeyHint(currentKV);
return KeyValueUtil.ensureKeyValue(this.filter.getNextCellHint((Cell)currentKV));
}
/**
* Old filter wrapper descendants will implement KV getNextKeyHint(KV) so we should call it.
*/
@Override
public Cell getNextCellHint(Cell currentKV) throws IOException {
// Old filters based off of this class will override KeyValue getNextKeyHint(KeyValue).
// Thus to maintain compatibility we need to call the old version.
return this.getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV));
}
@Override
@ -108,13 +126,26 @@ public class FilterWrapper extends Filter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
public ReturnCode filterKeyValue(Cell v) throws IOException {
return this.filter.filterKeyValue(v);
}
@Override
public KeyValue transform(KeyValue v) throws IOException {
return this.filter.transform(v);
public Cell transformCell(Cell v) throws IOException {
// Old filters based off of this class will override KeyValue transform(KeyValue).
// Thus to maintain compatibility we need to call the old version.
return transform(KeyValueUtil.ensureKeyValue(v));
}
/**
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
*
* This is for transition from 0.94 -> 0.96
*/
@Override
@Deprecated
public KeyValue transform(KeyValue currentKV) throws IOException {
return KeyValueUtil.ensureKeyValue(this.filter.transformCell(currentKV));
}
@Override
@ -123,18 +154,31 @@ public class FilterWrapper extends Filter {
}
@Override
public void filterRow(List<KeyValue> kvs) throws IOException {
//To fix HBASE-6429,
public void filterRowCells(List<Cell> kvs) throws IOException {
//To fix HBASE-6429,
//Filter with filterRow() returning true is incompatible with scan with limit
//1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
//2. filterRow() is merged with filterRow(kvs),
//so that to make all those row related filtering stuff in the same function.
this.filter.filterRow(kvs);
this.filter.filterRowCells(kvs);
if (!kvs.isEmpty() && this.filter.filterRow()) {
kvs.clear();
}
}
/**
* WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
*
* This is for transition from 0.94 -> 0.96
*/
@Override
@Deprecated
public void filterRow(List<KeyValue> kvs) throws IOException {
// This is only used internally, marked InterfaceAudience.private, and not used anywhere.
// We can get away with not implementing this.
throw new UnsupportedOperationException("filterRow(List<KeyValue>) should never be called");
}
@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);

View File

@ -18,15 +18,16 @@
*/
package org.apache.hadoop.hbase.filter;
import com.google.common.base.Preconditions;
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.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that will only return the first KV from each row.
@ -45,7 +46,8 @@ public class FirstKeyOnlyFilter extends FilterBase {
foundKV = false;
}
public ReturnCode filterKeyValue(KeyValue v) {
@Override
public ReturnCode filterKeyValue(Cell v) {
if(foundKV) return ReturnCode.NEXT_ROW;
foundKV = true;
return ReturnCode.INCLUDE;

View File

@ -22,7 +22,9 @@ import com.google.protobuf.ByteString;
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.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
@ -57,7 +59,8 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
this.qualifiers = qualifiers;
}
public ReturnCode filterKeyValue(KeyValue v) {
@Override
public ReturnCode filterKeyValue(Cell v) {
if (hasFoundKV()) {
return ReturnCode.NEXT_ROW;
} else if (hasOneMatchingQualifier(v)) {
@ -66,9 +69,11 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
return ReturnCode.INCLUDE;
}
private boolean hasOneMatchingQualifier(KeyValue v) {
private boolean hasOneMatchingQualifier(Cell v) {
for (byte[] q : qualifiers) {
if (v.matchingQualifier(q)) {
// TODO get rid of this by adding matching qualifier to interface.
KeyValue kv = KeyValueUtil.ensureKeyValue(v);
if (kv.matchingQualifier(q)) {
return true;
}
}

View File

@ -21,7 +21,9 @@ import com.google.protobuf.ByteString;
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.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
@ -72,8 +74,11 @@ public class FuzzyRowFilter extends FilterBase {
// TODO: possible improvement: save which fuzzy row key to use when providing a hint
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
byte[] rowKey = kv.getRow();
public ReturnCode filterKeyValue(Cell kv) {
// TODO add getRow() equivalent to Cell or change satisfies to take b[],o,l style args.
KeyValue v = KeyValueUtil.ensureKeyValue(kv);
byte[] rowKey = v.getRow();
// assigning "worst" result first and looking for better options
SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
@ -98,8 +103,11 @@ public class FuzzyRowFilter extends FilterBase {
}
@Override
public KeyValue getNextKeyHint(KeyValue currentKV) {
byte[] rowKey = currentKV.getRow();
public Cell getNextCellHint(Cell currentKV) {
// TODO make matching Column a cell method or CellUtil method.
KeyValue v = KeyValueUtil.ensureKeyValue(currentKV);
byte[] rowKey = v.getRow();
byte[] nextRowKey = null;
// Searching for the "smallest" row key that satisfies at least one fuzzy row key
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {

View File

@ -19,15 +19,18 @@
package org.apache.hadoop.hbase.filter;
import com.google.common.base.Preconditions;
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 java.util.ArrayList;
import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that will only return the key component of each KV (the value will
@ -45,8 +48,13 @@ public class KeyOnlyFilter extends FilterBase {
public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
@Override
public KeyValue transform(KeyValue kv) {
return kv.createKeyOnly(this.lenAsVal);
public Cell transformCell(Cell kv) {
// TODO Move to KeyValueUtil
// TODO make matching Column a cell method or CellUtil method.
KeyValue v = KeyValueUtil.ensureKeyValue(kv);
return v.createKeyOnly(this.lenAsVal);
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {

View File

@ -21,6 +21,7 @@ import com.google.protobuf.ByteString;
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;
@ -62,11 +63,11 @@ public class MultipleColumnPrefixFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
if (sortedPrefixes.size() == 0 || kv.getBuffer() == null) {
public ReturnCode filterKeyValue(Cell kv) {
if (sortedPrefixes.size() == 0 || kv.getQualifierArray() == null) {
return ReturnCode.INCLUDE;
} else {
return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
}
}
@ -152,9 +153,10 @@ public class MultipleColumnPrefixFilter extends FilterBase {
return this.sortedPrefixes.equals(other.sortedPrefixes);
}
public KeyValue getNextKeyHint(KeyValue kv) {
@Override
public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
}

View File

@ -19,13 +19,13 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A binary comparator which lexicographically compares against the specified
* byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.

View File

@ -44,7 +44,7 @@ import java.util.Stack;
* a filter object is constructed. This filter object is then wrapped
* in a scanner object which is then returned
* <p>
* This class addresses the HBASE-4168 JIRA. More documentaton on this
* This class addresses the HBASE-4168 JIRA. More documentation on this
* Filter Language can be found at: https://issues.apache.org/jira/browse/HBASE-4176
*/
@InterfaceAudience.Public
@ -242,8 +242,8 @@ public class ParseFilter {
}
try {
filterName = filterHashMap.get(filterName);
Class c = Class.forName(filterName);
Class[] argTypes = new Class [] {ArrayList.class};
Class<?> c = Class.forName(filterName);
Class<?>[] argTypes = new Class [] {ArrayList.class};
Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
return (Filter) m.invoke(null,filterArguments);
} catch (ClassNotFoundException e) {

View File

@ -19,17 +19,18 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import java.util.ArrayList;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter based on the column qualifier. It takes an
@ -59,10 +60,10 @@ public class QualifierFilter extends CompareFilter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell v) {
int qualifierLength = v.getQualifierLength();
if (qualifierLength > 0) {
if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
if (doCompare(this.compareOp, this.comparator, v.getQualifierArray(),
v.getQualifierOffset(), qualifierLength)) {
return ReturnCode.SKIP;
}
@ -71,7 +72,7 @@ public class QualifierFilter extends CompareFilter {
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
ArrayList<?> arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
return new QualifierFilter(compareOp, comparator);

View File

@ -19,14 +19,15 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.util.Random;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.util.Random;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that includes rows based on a chance.
@ -71,7 +72,7 @@ public class RandomRowFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell v) {
if (filterOutRow) {
return ReturnCode.NEXT_ROW;
}

View File

@ -19,17 +19,18 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import java.util.ArrayList;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter based on the key. It takes an operator
@ -65,7 +66,7 @@ public class RowFilter extends CompareFilter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell v) {
if(this.filterOutRow) {
return ReturnCode.NEXT_ROW;
}
@ -86,6 +87,7 @@ public class RowFilter extends CompareFilter {
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
@SuppressWarnings("rawtypes") // for arguments
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);

View File

@ -22,7 +22,9 @@ package org.apache.hadoop.hbase.filter;
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.KeyValueUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -100,10 +102,11 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
}
// Here we remove from row all key values from testing column
public void filterRow(List<KeyValue> kvs) {
Iterator it = kvs.iterator();
@Override
public void filterRowCells(List<Cell> kvs) {
Iterator<? extends Cell> it = kvs.iterator();
while (it.hasNext()) {
KeyValue kv = (KeyValue)it.next();
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)) {

View File

@ -26,7 +26,9 @@ 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.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@ -166,7 +168,11 @@ public class SingleColumnValueFilter extends FilterBase {
return columnQualifier;
}
public ReturnCode filterKeyValue(KeyValue keyValue) {
@Override
public ReturnCode filterKeyValue(Cell c) {
// TODO get rid of this.
KeyValue keyValue = KeyValueUtil.ensureKeyValue(c);
// System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue()));
if (this.matchedColumn) {
// We already found and matched the single column, all keys now pass

View File

@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A wrapper filter that filters an entire row if any of the KeyValue checks do
* A wrapper filter that filters an entire row if any of the Cell checks do
* not pass.
* <p>
* For example, if all columns in a row represent weights of different things,
@ -45,7 +45,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
* new BinaryComparator(Bytes.toBytes(0))));
* </code>
* Any row which contained a column whose value was 0 will be filtered out
* (since ValueFilter will not pass that KeyValue).
* (since ValueFilter will not pass that Cell).
* Without this filter, the other non-zero valued columns in the row would still
* be emitted.
*/
@ -74,15 +74,15 @@ public class SkipFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFR(c != ReturnCode.INCLUDE);
return c;
}
@Override
public KeyValue transform(KeyValue v) throws IOException {
return filter.transform(v);
public Cell transformCell(Cell v) throws IOException {
return filter.transformCell(v);
}
public boolean filterRow() {

View File

@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@ -88,7 +88,7 @@ public class TimestampsFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(Cell v) {
if (this.timestamps.contains(v.getTimestamp())) {
return ReturnCode.INCLUDE;
} else if (v.getTimestamp() < minTimeStamp) {

View File

@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@ -58,8 +58,8 @@ public class ValueFilter extends CompareFilter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
public ReturnCode filterKeyValue(Cell v) {
if (doCompare(this.compareOp, this.comparator, v.getValueArray(),
v.getValueOffset(), v.getValueLength())) {
return ReturnCode.SKIP;
}
@ -67,6 +67,7 @@ public class ValueFilter extends CompareFilter {
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
@SuppressWarnings("rawtypes") // for arguments
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);

View File

@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@ -33,7 +33,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
/**
* A wrapper filter that returns true from {@link #filterAllRemaining()} as soon
* as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)},
* {@link Filter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)},
* {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)},
* {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or
* {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods
* returns true.
@ -73,15 +73,15 @@ public class WhileMatchFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFAR(c != ReturnCode.INCLUDE);
return c;
}
@Override
public KeyValue transform(KeyValue v) throws IOException {
return filter.transform(v);
public Cell transformCell(Cell v) throws IOException {
return filter.transformCell(v);
}
@Override

View File

@ -73,73 +73,73 @@ public class TestPayloadCarryingRpcController {
@Override
public byte[] getRowArray() {
// TODO Auto-generated method stub
// unused
return null;
}
@Override
public int getRowOffset() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public short getRowLength() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public byte[] getFamilyArray() {
// TODO Auto-generated method stub
// unused
return null;
}
@Override
public int getFamilyOffset() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public byte getFamilyLength() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public byte[] getQualifierArray() {
// TODO Auto-generated method stub
// unused
return null;
}
@Override
public int getQualifierOffset() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public int getQualifierLength() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public long getTimestamp() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public byte getTypeByte() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public long getMvccVersion() {
// TODO Auto-generated method stub
// unused
return 0;
}
@ -160,19 +160,43 @@ public class TestPayloadCarryingRpcController {
@Override
public int getTagsOffset() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public short getTagsLength() {
// TODO Auto-generated method stub
// unused
return 0;
}
@Override
public byte[] getTagsArray() {
// TODO Auto-generated method stub
// unused
return null;
}
@Override
public byte[] getValue() {
// unused
return null;
}
@Override
public byte[] getFamily() {
// unused
return null;
}
@Override
public byte[] getQualifier() {
// unused
return null;
}
@Override
public byte[] getRow() {
// unused
return null;
}
};

View File

@ -41,8 +41,10 @@ import org.apache.hadoop.classification.InterfaceStability;
* qualifier. Less intuitively, it will then treat the greater timestamp as the lesser value with
* the goal of sorting newer cells first.
* <p/>
* This interface does not include methods that allocate new byte[]'s such as those used in client
* or debugging code. These should be placed in a sub-interface or the {@link CellUtil} class.
* This interface should not include methods that allocate new byte[]'s such as those used in client
* or debugging code. These users should use the methods found in the {@link CellUtil} class.
* Currently for to minimize the impact of existing applications moving between 0.94 and 0.96, we
* include the costly helper methods marked as deprecated.
* <p/>
* Cell implements Comparable<Cell> which is only meaningful when comparing to other keys in the
* same table. It uses CellComparator which does not work on the -ROOT- and .META. tables.
@ -182,5 +184,44 @@ public interface Cell {
* @return the total length of the tags in the Cell.
*/
short getTagsLength();
/**
* WARNING do not use, expensive. This gets an arraycopy of the cell's value.
*
* Added to ease transition from 0.94 -> 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#getValueArray(Cell)}
*/
@Deprecated
byte[] getValue();
/**
* WARNING do not use, expensive. This gets an arraycopy of the cell's family.
*
* Added to ease transition from 0.94 -> 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#getFamilyArray(Cell)}
*/
@Deprecated
byte[] getFamily();
/**
* WARNING do not use, expensive. This gets an arraycopy of the cell's qualifier.
*
* Added to ease transition from 0.94 -> 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#getQualifierArray(Cell)}
*/
@Deprecated
byte[] getQualifier();
/**
* WARNING do not use, expensive. this gets an arraycopy of the cell's row.
*
* Added to ease transition from 0.94 -> 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#getRowByte(Cell, int)}
*/
@Deprecated
byte[] getRow();
}

View File

@ -27,6 +27,7 @@ import java.util.NavigableMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.util.ByteRange;
import org.apache.hadoop.hbase.util.Bytes;
@ -268,6 +269,41 @@ public final class CellUtil {
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);
}
public static boolean matchingFamily(final Cell left, final Cell right) {
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
}
public static boolean matchingFamily(final Cell left, final byte[] buf) {
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
buf, 0, buf.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());
}
public static boolean matchingQualifier(final Cell left, final byte[] buf) {
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
buf, 0, buf.length);
}
public static boolean matchingValue(final Cell left, final Cell right) {
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
right.getValueArray(), right.getValueOffset(), right.getValueLength());
}
public static boolean matchingValue(final Cell left, final byte[] buf) {
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}
@ -277,6 +313,10 @@ public final class CellUtil {
return KeyValue.isDelete(cell.getTypeByte());
}
public static boolean isDeleteFamily(final Cell cell) {
return cell.getTypeByte() == Type.DeleteFamily.getCode();
}
/**
* @param cell
* @return Estimate of the <code>cell</code> size in bytes.

View File

@ -26,7 +26,6 @@ import java.io.DataOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Comparator;
@ -36,7 +35,6 @@ import java.util.Map;
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.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@ -46,7 +44,10 @@ import org.apache.hadoop.io.RawComparator;
import com.google.common.primitives.Longs;
/**
* An HBase Key/Value. This is the fundamental HBase Type.
* An HBase Key/Value. This is the fundamental HBase Type.
* <p>
* HBase applications and users should use the Cell interface and avoid directly using KeyValue
* and member functions not defined in Cell.
* <p>
* If being used client-side, the primary methods to access individual fields are {@link #getRow()},
* {@link #getFamily()}, {@link #getQualifier()}, {@link #getTimestamp()}, and {@link #getValue()}.
@ -66,8 +67,7 @@ import com.google.common.primitives.Longs;
* <code>Integer.MAX_SIZE</code>. The column does not contain the family/qualifier delimiter,
* {@link #COLUMN_FAMILY_DELIMITER}
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@InterfaceAudience.Private
public class KeyValue implements Cell, HeapSize, Cloneable {
static final Log LOG = LogFactory.getLog(KeyValue.class);
@ -438,6 +438,14 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
this.offset = 0;
}
public KeyValue(Cell c) {
this(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(),
c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
c.getTimestamp(), Type.codeToType(c.getTypeByte()),
c.getValueArray(), c.getValueOffset(), c.getValueLength());
}
/**
* Create an empty byte[] representing a KeyValue
* All lengths are preset and can be filled in later.
@ -1056,34 +1064,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* save on allocations.
* @return Value in a new byte array.
*/
@Deprecated // use CellUtil.getValueArray()
public byte [] getValue() {
int o = getValueOffset();
int l = getValueLength();
byte [] result = new byte[l];
System.arraycopy(getBuffer(), o, result, 0, l);
return result;
}
/**
* Returns the value wrapped in a new <code>ByteBuffer</code>.
*
* @return the value
*/
public ByteBuffer getValueAsByteBuffer() {
return ByteBuffer.wrap(getBuffer(), getValueOffset(), getValueLength());
}
/**
* Loads this object's value into the provided <code>ByteBuffer</code>.
* <p>
* Does not clear or flip the buffer.
*
* @param dst the buffer where to write the value
*
* @throws BufferOverflowException if there is insufficient space remaining in the buffer
*/
public void loadValue(ByteBuffer dst) throws BufferOverflowException {
dst.put(getBuffer(), getValueOffset(), getValueLength());
return CellUtil.getValueArray(this);
}
/**
@ -1094,12 +1077,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* lengths instead.
* @return Row in a new byte array.
*/
@Deprecated // use CellUtil.getRowArray()
public byte [] getRow() {
int o = getRowOffset();
short l = getRowLength();
byte result[] = new byte[l];
System.arraycopy(getBuffer(), o, result, 0, l);
return result;
return CellUtil.getRowArray(this);
}
/**
@ -1141,6 +1121,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
* KeyValue type.
*/
@Deprecated // use CellUtil#isDelete
public boolean isDelete() {
return KeyValue.isDelete(getType());
}
@ -1184,12 +1165,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* lengths instead.
* @return Returns family. Makes a copy.
*/
@Deprecated // use CellUtil.getFamilyArray
public byte [] getFamily() {
int o = getFamilyOffset();
int l = getFamilyLength(o);
byte [] result = new byte[l];
System.arraycopy(this.bytes, o, result, 0, l);
return result;
return CellUtil.getFamilyArray(this);
}
/**
@ -1201,12 +1179,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* Use {@link #getBuffer()} with appropriate offsets and lengths instead.
* @return Returns qualifier. Makes a copy.
*/
@Deprecated // use CellUtil.getQualifierArray
public byte [] getQualifier() {
int o = getQualifierOffset();
int l = getQualifierLength();
byte [] result = new byte[l];
System.arraycopy(this.bytes, o, result, 0, l);
return result;
return CellUtil.getQualifierArray(this);
}
//---------------------------------------------------------------------------
@ -1300,6 +1275,8 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
/**
* Creates a new KeyValue that only contains the key portion (the value is
* set to be null).
*
* TODO only used by KeyOnlyFilter -- move there.
* @param lenAsVal replace value with the actual value length (false=empty)
*/
public KeyValue createKeyOnly(boolean lenAsVal) {
@ -2467,4 +2444,5 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
public byte[] getTagsArray() {
throw new UnsupportedOperationException("Not implememnted");
}
}

View File

@ -80,11 +80,11 @@ public class KeyValueTestUtil {
* @param kvCollection2
* @return true if KeyValues from kvCollection2 are contained in kvCollection1
*/
public static boolean containsIgnoreMvccVersion(Collection<KeyValue> kvCollection1,
Collection<KeyValue> kvCollection2) {
for (KeyValue kv1 : kvCollection1) {
public static boolean containsIgnoreMvccVersion(Collection<? extends Cell> kvCollection1,
Collection<? extends Cell> kvCollection2) {
for (Cell kv1 : kvCollection1) {
boolean found = false;
for (KeyValue kv2 : kvCollection2) {
for (Cell kv2 : kvCollection2) {
if (CellComparator.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
}
if (!found) return false;

View File

@ -26,6 +26,7 @@ import java.util.Iterator;
import org.apache.commons.lang.NotImplementedException;
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.io.compress.Compression;
@ -73,7 +74,7 @@ public class EncodedDataBlock {
* @param headerSize header size of the block.
* @return Forwards sequential iterator.
*/
public Iterator<KeyValue> getIterator(int headerSize) {
public Iterator<Cell> getIterator(int headerSize) {
final int rawSize = rawKVs.length;
byte[] encodedDataWithHeader = getEncodedData();
int bytesToSkip = headerSize + Bytes.SIZEOF_SHORT;
@ -81,7 +82,7 @@ public class EncodedDataBlock {
bytesToSkip, encodedDataWithHeader.length - bytesToSkip);
final DataInputStream dis = new DataInputStream(bais);
return new Iterator<KeyValue>() {
return new Iterator<Cell>() {
private ByteBuffer decompressedData = null;
@Override
@ -93,7 +94,7 @@ public class EncodedDataBlock {
}
@Override
public KeyValue next() {
public Cell next() {
if (decompressedData == null) {
try {
decompressedData = dataBlockEncoder.decodeKeyValues(

View File

@ -26,10 +26,11 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
@ -132,9 +133,9 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
// filter and having necessary column(s).
scanner = region.getScanner(scan);
while (hasMore) {
List<List<KeyValue>> deleteRows = new ArrayList<List<KeyValue>>(rowBatchSize);
List<List<Cell>> deleteRows = new ArrayList<List<Cell>>(rowBatchSize);
for (int i = 0; i < rowBatchSize; i++) {
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
hasMore = scanner.next(results);
if (results.size() > 0) {
deleteRows.add(results);
@ -147,7 +148,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
if (deleteRows.size() > 0) {
Mutation[] deleteArr = new Mutation[deleteRows.size()];
int i = 0;
for (List<KeyValue> deleteRow : deleteRows) {
for (List<Cell> deleteRow : deleteRows) {
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
}
OperationStatus[] opStatus = region.batchMutate(deleteArr);
@ -188,7 +189,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
done.run(result);
}
private Delete createDeleteMutation(List<KeyValue> deleteRow, DeleteType deleteType,
private Delete createDeleteMutation(List<Cell> deleteRow, DeleteType deleteType,
Long timestamp) {
long ts;
if (timestamp == null) {
@ -197,19 +198,19 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
ts = timestamp;
}
// We just need the rowkey. Get it from 1st KV.
byte[] row = deleteRow.get(0).getRow();
byte[] row = CellUtil.getRowArray(deleteRow.get(0));
Delete delete = new Delete(row, ts);
if (deleteType == DeleteType.FAMILY) {
Set<byte[]> families = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
for (KeyValue kv : deleteRow) {
if (families.add(kv.getFamily())) {
delete.deleteFamily(kv.getFamily(), ts);
for (Cell kv : deleteRow) {
if (families.add(CellUtil.getFamilyArray(kv))) {
delete.deleteFamily(CellUtil.getFamilyArray(kv), ts);
}
}
} else if (deleteType == DeleteType.COLUMN) {
Set<Column> columns = new HashSet<Column>();
for (KeyValue kv : deleteRow) {
Column column = new Column(kv.getFamily(), kv.getQualifier());
for (Cell kv : deleteRow) {
Column column = new Column(CellUtil.getFamilyArray(kv), CellUtil.getQualifierArray(kv));
if (columns.add(column)) {
// Making deleteColumns() calls more than once for the same cf:qualifier is not correct
// Every call to deleteColumns() will add a new KV to the familymap which will finally
@ -224,14 +225,14 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
// the scan fetched will get deleted.
int noOfVersionsToDelete = 0;
if (timestamp == null) {
for (KeyValue kv : deleteRow) {
delete.deleteColumn(kv.getFamily(), kv.getQualifier(), kv.getTimestamp());
for (Cell kv : deleteRow) {
delete.deleteColumn(CellUtil.getFamilyArray(kv), CellUtil.getQualifierArray(kv), kv.getTimestamp());
noOfVersionsToDelete++;
}
} else {
Set<Column> columns = new HashSet<Column>();
for (KeyValue kv : deleteRow) {
Column column = new Column(kv.getFamily(), kv.getQualifier());
for (Cell kv : deleteRow) {
Column column = new Column(CellUtil.getFamilyArray(kv), CellUtil.getQualifierArray(kv));
// Only one version of particular column getting deleted.
if (columns.add(column)) {
delete.deleteColumn(column.family, column.qualifier, ts);

View File

@ -18,12 +18,14 @@
package org.apache.hadoop.hbase.coprocessor.example;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@ -34,9 +36,9 @@ import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
/**
* Sample coprocessor endpoint exposing a Service interface for counting rows and key values.
@ -73,14 +75,14 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(scan);
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
boolean hasMore = false;
byte[] lastRow = null;
long count = 0;
do {
hasMore = scanner.next(results);
for (KeyValue kv : results) {
byte[] currentRow = kv.getRow();
for (Cell kv : results) {
byte[] currentRow = CellUtil.getRowArray(kv);
if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
lastRow = currentRow;
count++;
@ -113,12 +115,12 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(new Scan());
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
boolean hasMore = false;
long count = 0;
do {
hasMore = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
count++;
}
results.clear();

View File

@ -25,11 +25,12 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MediumTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
@ -39,22 +40,18 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.Ignore;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
@ -276,17 +273,17 @@ public class TestBulkDeleteProtocol {
scan.setMaxVersions();
for (Result result : ht.getScanner(scan)) {
assertEquals(3, result.getFamilyMap(FAMILY1).size());
List<KeyValue> column = result.getColumn(FAMILY1, QUALIFIER1);
List<Cell> column = result.getColumn(FAMILY1, QUALIFIER1);
assertEquals(1, column.size());
assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
column = result.getColumn(FAMILY1, QUALIFIER2);
assertEquals(1, column.size());
assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
column = result.getColumn(FAMILY1, QUALIFIER3);
assertEquals(1, column.size());
assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
rows++;
}
assertEquals(100, rows);
@ -330,10 +327,10 @@ public class TestBulkDeleteProtocol {
assertEquals(3, result.getFamilyMap(FAMILY1).size());
assertEquals(3, result.getColumn(FAMILY1, QUALIFIER1).size());
assertEquals(3, result.getColumn(FAMILY1, QUALIFIER2).size());
List<KeyValue> column = result.getColumn(FAMILY1, QUALIFIER3);
List<Cell> column = result.getColumn(FAMILY1, QUALIFIER3);
assertEquals(2, column.size());
assertTrue(Bytes.equals("v3".getBytes(), column.get(0).getValue()));
assertTrue(Bytes.equals("v1".getBytes(), column.get(1).getValue()));
assertTrue(CellUtil.matchingValue(column.get(0), "v3".getBytes()));
assertTrue(CellUtil.matchingValue(column.get(1), "v1".getBytes()));
rows++;
}
assertEquals(100, rows);
@ -410,14 +407,14 @@ public class TestBulkDeleteProtocol {
scan1.setMaxVersions();
for (Result res : ht.getScanner(scan1)) {
assertEquals(3, res.getFamilyMap(FAMILY1).size());
List<KeyValue> column = res.getColumn(FAMILY1, QUALIFIER1);
List<Cell> column = res.getColumn(FAMILY1, QUALIFIER1);
assertEquals(2, column.size());
assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue()));
assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue()));
assertTrue(CellUtil.matchingValue(column.get(0), "v4".getBytes()));
assertTrue(CellUtil.matchingValue(column.get(1), "v3".getBytes()));
column = res.getColumn(FAMILY1, QUALIFIER2);
assertEquals(2, column.size());
assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue()));
assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue()));
assertTrue(CellUtil.matchingValue(column.get(0), "v4".getBytes()));
assertTrue(CellUtil.matchingValue(column.get(1), "v3".getBytes()));
assertEquals(4, res.getColumn(FAMILY1, QUALIFIER3).size());
rows++;
}

View File

@ -18,6 +18,8 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import static org.junit.Assert.assertEquals;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
@ -30,6 +32,8 @@ import java.util.Set;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
@ -65,8 +69,6 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
/**
* Test Bulk Load and MR on a distributed cluster.
* It starts an MR job that creates linked chains
@ -468,7 +470,8 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
long chainId = Bytes.toLong(entry.getKey());
long next = Bytes.toLong(entry.getValue());
long order = Bytes.toLong(value.getColumn(SORT_FAM, entry.getKey()).get(0).getValue());
Cell c = value.getColumn(SORT_FAM, entry.getKey()).get(0);
long order = Bytes.toLong(CellUtil.getValueArray(c));
context.write(new LinkKey(chainId, order), new LinkChain(longRk, next));
}
}

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import static java.lang.String.format;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
@ -31,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
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.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -62,11 +68,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static java.lang.String.format;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
/**
* Validate ImportTsv + LoadIncrementalHFiles on a distributed cluster.
*/
@ -153,7 +154,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool {
Iterator<KeyValue> expectedIt = simple_expected.iterator();
while (resultsIt.hasNext() && expectedIt.hasNext()) {
Result r = resultsIt.next();
for (KeyValue actual : r.raw()) {
for (Cell actual : r.raw()) {
assertTrue(
"Ran out of expected values prematurely!",
expectedIt.hasNext());

View File

@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hbase.test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Random;
import java.util.Set;
@ -24,13 +27,13 @@ import java.util.UUID;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import com.google.common.collect.Sets;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.logging.Log;
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.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -38,7 +41,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.IntegrationTests;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@ -65,8 +67,7 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Sets;
/**
* A large test which loads a lot of data that has internal references, and
@ -247,12 +248,12 @@ public class IntegrationTestLoadAndVerify extends IntegrationTestBase {
throws IOException, InterruptedException {
BytesWritable bwKey = new BytesWritable(key.get());
BytesWritable bwVal = new BytesWritable();
for (KeyValue kv : value.list()) {
for (Cell kv : value.list()) {
if (Bytes.compareTo(TEST_QUALIFIER, 0, TEST_QUALIFIER.length,
kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
context.write(bwKey, EMPTY);
} else {
bwVal.set(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
bwVal.set(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
context.write(bwVal, bwKey);
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.decode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
@ -183,6 +184,26 @@ public class PrefixTreeCell implements Cell, Comparable<Cell> {
return type.getCode();
}
/* Deprecated methods pushed into the Cell interface */
@Override
public byte[] getValue() {
return CellUtil.getValueArray(this);
}
@Override
public byte[] getFamily() {
return CellUtil.getFamilyArray(this);
}
@Override
public byte[] getQualifier() {
return CellUtil.getQualifierArray(this);
}
@Override
public byte[] getRow() {
return CellUtil.getRowArray(this);
}
/************************* helper methods *************************/
@ -208,4 +229,5 @@ public class PrefixTreeCell implements Cell, Comparable<Cell> {
public byte[] getTagsArray() {
throw new UnsupportedOperationException("Not implemented");
}
}

View File

@ -28,11 +28,10 @@ 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.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
@ -84,7 +83,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
T temp;
Scan scan = ProtobufUtil.toScan(request.getScan());
scanner = env.getRegion().getScanner(scan);
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
byte[] colFamily = scan.getFamilies()[0];
NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
byte[] qualifier = null;
@ -95,7 +94,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
temp = ci.getValue(colFamily, qualifier, kv);
max = (max == null || (temp != null && ci.compare(temp, max) > 0)) ? temp : max;
}
@ -138,7 +137,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
T temp;
Scan scan = ProtobufUtil.toScan(request.getScan());
scanner = env.getRegion().getScanner(scan);
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
byte[] colFamily = scan.getFamilies()[0];
NavigableSet<byte[]> qualifiers = scan.getFamilyMap().get(colFamily);
byte[] qualifier = null;
@ -148,7 +147,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
temp = ci.getValue(colFamily, qualifier, kv);
min = (min == null || (temp != null && ci.compare(temp, min) < 0)) ? temp : min;
}
@ -197,11 +196,11 @@ extends AggregateService implements CoprocessorService, Coprocessor {
if (qualifiers != null && !qualifiers.isEmpty()) {
qualifier = qualifiers.pollFirst();
}
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
temp = ci.getValue(colFamily, qualifier, kv);
if (temp != null)
sumVal = ci.add(sumVal, ci.castToReturnType(temp));
@ -236,7 +235,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
RpcCallback<AggregateResponse> done) {
AggregateResponse response = null;
long counter = 0l;
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
InternalScanner scanner = null;
try {
Scan scan = ProtobufUtil.toScan(request.getScan());
@ -305,13 +304,13 @@ extends AggregateService implements CoprocessorService, Coprocessor {
if (qualifiers != null && !qualifiers.isEmpty()) {
qualifier = qualifiers.pollFirst();
}
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
boolean hasMoreRows = false;
do {
results.clear();
hasMoreRows = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
sumVal = ci.add(sumVal, ci.castToReturnType(ci.getValue(colFamily,
qualifier, kv)));
}
@ -364,14 +363,14 @@ extends AggregateService implements CoprocessorService, Coprocessor {
if (qualifiers != null && !qualifiers.isEmpty()) {
qualifier = qualifiers.pollFirst();
}
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
boolean hasMoreRows = false;
do {
tempVal = null;
hasMoreRows = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily,
qualifier, kv)));
}
@ -429,7 +428,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
// if weighted median is requested, get qualifier for the weight column
weightQualifier = qualifiers.pollLast();
}
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
boolean hasMoreRows = false;
@ -437,7 +436,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
tempVal = null;
tempWeight = null;
hasMoreRows = scanner.next(results);
for (KeyValue kv : results) {
for (Cell kv : results) {
tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily,
valQualifier, kv)));
if (weightQualifier != null) {

View File

@ -15,27 +15,29 @@
package org.apache.hadoop.hbase.coprocessor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import com.google.common.collect.ImmutableList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
@ -48,8 +50,9 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Pair;
import org.mortbay.log.Log;
import java.io.IOException;
import com.google.common.collect.ImmutableList;
/**
* An abstract class that implements RegionObserver.
@ -209,15 +212,56 @@ public abstract class BaseRegionObserver implements RegionObserver {
}
@Override
public void preGet(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final List<KeyValue> results) throws IOException {
public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final List<Cell> results) throws IOException {
// By default we are executing the deprecated preGet to support legacy RegionObservers
// We may use the results coming in and we may return the results going out.
List<KeyValue> kvs = new ArrayList<KeyValue>(results.size());
for (Cell c : results) {
kvs.add(KeyValueUtil.ensureKeyValue(c));
}
preGet(e, get, kvs);
results.clear();
results.addAll(kvs);
}
/**
* WARNING: please override preGetOp instead of this method. This is to maintain some
* compatibility and to ease the transition from 0.94 -> 0.96. It is super inefficient!
*/
@Deprecated
@Override
public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> result)
throws IOException {
}
@Override
public void postGet(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final List<KeyValue> results) throws IOException {
public void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final List<Cell> results) throws IOException {
// By default we are executing the deprecated preGet to support legacy RegionObservers
// We may use the results coming in and we may return the results going out.
List<KeyValue> kvs = new ArrayList<KeyValue>(results.size());
for (Cell c : results) {
kvs.add(KeyValueUtil.ensureKeyValue(c));
}
postGet(e, get, kvs);
results.clear();
results.addAll(kvs);
}
/**
* WARNING: please override postGetOp instead of this method. This is to maintain some
* compatibility and to ease the transition from 0.94 -> 0.96. It is super inefficient!
*/
@Deprecated
@Override
public void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> result)
throws IOException {
}
@Override
public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final boolean exists) throws IOException {

View File

@ -21,6 +21,7 @@ import java.util.NavigableSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
@ -429,6 +430,15 @@ public interface RegionObserver extends Coprocessor {
* is not bypassed.
* @throws IOException if an error occurred on the coprocessor
*/
void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<Cell> result)
throws IOException;
/**
* WARNING: please override preGetOp instead of this method. This is to maintain some
* compatibility and to ease the transition from 0.94 -> 0.96.
*/
@Deprecated
void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> result)
throws IOException;
@ -443,6 +453,15 @@ public interface RegionObserver extends Coprocessor {
* @param result the result to return to the client, modify as necessary
* @throws IOException if an error occurred on the coprocessor
*/
void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<Cell> result)
throws IOException;
/**
* WARNING: please override postGetOp instead of this method. This is to maintain some
* compatibility and to ease the transition from 0.94 -> 0.96.
*/
@Deprecated
void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> result)
throws IOException;

View File

@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.mapred;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Map;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MapReduceBase;
@ -115,12 +116,12 @@ implements TableMap<ImmutableBytesWritable,Result> {
ArrayList<byte[]> foundList = new ArrayList<byte[]>();
int numCols = columns.length;
if (numCols > 0) {
for (KeyValue value: r.list()) {
byte [] column = KeyValue.makeColumn(value.getFamily(),
value.getQualifier());
for (Cell value: r.list()) {
byte [] column = KeyValue.makeColumn(CellUtil.getFamilyArray(value),
CellUtil.getQualifierArray(value));
for (int i = 0; i < numCols; i++) {
if (Bytes.equals(column, columns[i])) {
foundList.add(value.getValue());
foundList.add(CellUtil.getValueArray(value));
break;
}
}

View File

@ -26,8 +26,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.CompareFilter;
@ -115,9 +116,9 @@ public class CellCounter {
context.getCounter(Counters.ROWS).increment(1);
context.write(new Text("Total ROWS"), new IntWritable(1));
for (KeyValue value : values.list()) {
currentRowKey = Bytes.toStringBinary(value.getRow());
String thisRowFamilyName = Bytes.toStringBinary(value.getFamily());
for (Cell value : values.list()) {
currentRowKey = Bytes.toStringBinary(CellUtil.getRowArray(value));
String thisRowFamilyName = Bytes.toStringBinary(CellUtil.getFamilyArray(value));
if (!thisRowFamilyName.equals(currentFamilyName)) {
currentFamilyName = thisRowFamilyName;
context.getCounter("CF", thisRowFamilyName).increment(1);
@ -126,7 +127,7 @@ public class CellCounter {
context.write(new Text(thisRowFamilyName), new IntWritable(1));
}
String thisRowQualifierName = thisRowFamilyName + separator
+ Bytes.toStringBinary(value.getQualifier());
+ Bytes.toStringBinary(CellUtil.getQualifierArray(value));
if (!thisRowQualifierName.equals(currentQualifierName)) {
currentQualifierName = thisRowQualifierName;
context.getCounter("CFQL", thisRowQualifierName).increment(1);

View File

@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@ -107,12 +109,12 @@ extends TableMapper<ImmutableBytesWritable,Result> implements Configurable {
ArrayList<byte[]> foundList = new ArrayList<byte[]>();
int numCols = columns.length;
if (numCols > 0) {
for (KeyValue value: r.list()) {
byte [] column = KeyValue.makeColumn(value.getFamily(),
value.getQualifier());
for (Cell value: r.list()) {
byte [] column = KeyValue.makeColumn(CellUtil.getFamilyArray(value),
CellUtil.getQualifierArray(value));
for (int i = 0; i < numCols; i++) {
if (Bytes.equals(column, columns[i])) {
foundList.add(value.getValue());
foundList.add(CellUtil.getValueArray(value));
break;
}
}

View File

@ -34,8 +34,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HTable;
@ -90,12 +93,12 @@ public class Import {
Context context)
throws IOException {
try {
for (KeyValue kv : value.raw()) {
for (Cell kv : value.raw()) {
kv = filterKv(kv);
// skip if we filtered it out
if (kv == null) continue;
context.write(row, convertKv(kv, cfRenameMap));
// TODO get rid of ensureKeyValue
context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
}
} catch (InterruptedException e) {
e.printStackTrace();
@ -140,14 +143,14 @@ public class Import {
throws IOException, InterruptedException {
Put put = null;
Delete delete = null;
for (KeyValue kv : result.raw()) {
for (Cell kv : result.raw()) {
kv = filterKv(kv);
// skip if we filter it out
if (kv == null) continue;
kv = convertKv(kv, cfRenameMap);
// Deletes and Puts are gathered and written when finished
if (kv.isDelete()) {
if (CellUtil.isDelete(kv)) {
if (delete == null) {
delete = new Delete(key.get());
}
@ -245,7 +248,7 @@ public class Import {
* @return <tt>null</tt> if the key should not be written, otherwise returns the original
* {@link KeyValue}
*/
private static KeyValue filterKv(KeyValue kv) throws IOException {
private static Cell filterKv(Cell kv) throws IOException {
// apply the filter and skip this kv if the filter doesn't apply
if (filter != null) {
Filter.ReturnCode code = filter.filterKeyValue(kv);
@ -261,23 +264,23 @@ public class Import {
}
// helper: create a new KeyValue based on CF rename map
private static KeyValue convertKv(KeyValue kv, Map<byte[], byte[]> cfRenameMap) {
private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
if(cfRenameMap != null) {
// If there's a rename mapping for this CF, create a new KeyValue
byte[] newCfName = cfRenameMap.get(kv.getFamily());
byte[] newCfName = cfRenameMap.get(CellUtil.getFamilyArray(kv));
if(newCfName != null) {
kv = new KeyValue(kv.getBuffer(), // row buffer
kv = new KeyValue(kv.getRowArray(), // row buffer
kv.getRowOffset(), // row offset
kv.getRowLength(), // row length
newCfName, // CF buffer
0, // CF offset
newCfName.length, // CF length
kv.getBuffer(), // qualifier buffer
kv.getQualifierArray(), // qualifier buffer
kv.getQualifierOffset(), // qualifier offset
kv.getQualifierLength(), // qualifier length
kv.getTimestamp(), // timestamp
KeyValue.Type.codeToType(kv.getType()), // KV Type
kv.getBuffer(), // value buffer
KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type
kv.getValueArray(), // value buffer
kv.getValueOffset(), // value offset
kv.getValueLength()); // value length
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@ -94,11 +95,11 @@ public class TableNamespaceManager {
ResultScanner scanner = table.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try {
for(Result result : scanner) {
byte[] val = CellUtil.getValueArray(result.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
NamespaceDescriptor ns =
ProtobufUtil.toNamespaceDescriptor(
HBaseProtos.NamespaceDescriptor.parseFrom(
result.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES).getValue()));
HBaseProtos.NamespaceDescriptor.parseFrom(val));
zkNamespaceManager.update(ns);
}
} finally {
@ -112,11 +113,11 @@ public class TableNamespaceManager {
if (res.isEmpty()) {
return null;
}
byte[] val = CellUtil.getValueArray(res.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
return
ProtobufUtil.toNamespaceDescriptor(
HBaseProtos.NamespaceDescriptor.parseFrom(
res.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES).getValue()));
HBaseProtos.NamespaceDescriptor.parseFrom(val));
}
public synchronized void create(NamespaceDescriptor ns) throws IOException {
@ -185,10 +186,10 @@ public class TableNamespaceManager {
ResultScanner scanner = table.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
try {
for(Result r : scanner) {
ret.add(ProtobufUtil.toNamespaceDescriptor(
HBaseProtos.NamespaceDescriptor.parseFrom(
r.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES).getValue())));
byte[] val = CellUtil.getValueArray(r.getColumnLatest(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
ret.add(ProtobufUtil.toNamespaceDescriptor(
HBaseProtos.NamespaceDescriptor.parseFrom(val)));
}
} finally {
scanner.close();

View File

@ -66,6 +66,7 @@ import org.apache.hadoop.fs.FileStatus;
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.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -1825,7 +1826,7 @@ public class HRegion implements HeapSize { // , Writable{
get.setMaxVersions(count);
get.addColumn(family, qual);
List<KeyValue> result = get(get, false);
List<Cell> result = get(get, false);
if (result.size() < count) {
// Nothing to delete
@ -1835,7 +1836,7 @@ public class HRegion implements HeapSize { // , Writable{
if (result.size() > count) {
throw new RuntimeException("Unexpected size: " + result.size());
}
KeyValue getkv = result.get(count - 1);
KeyValue getkv = KeyValueUtil.ensureKeyValue(result.get(count - 1));
Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(),
getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
} else {
@ -2354,7 +2355,7 @@ public class HRegion implements HeapSize { // , Writable{
RowLock rowLock = getRowLock(get.getRow());
// wait for all previous transactions to complete (with lock held)
mvcc.completeMemstoreInsert(mvcc.beginMemstoreInsert());
List<KeyValue> result;
List<Cell> result;
try {
result = get(get, false);
@ -2363,12 +2364,12 @@ public class HRegion implements HeapSize { // , Writable{
boolean matches = false;
if (result.size() == 0 && valueIsNull) {
matches = true;
} else if (result.size() > 0 && result.get(0).getValue().length == 0 &&
} else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
valueIsNull) {
matches = true;
} else if (result.size() == 1 && !valueIsNull) {
KeyValue kv = result.get(0);
int compareResult = comparator.compareTo(kv.getBuffer(),
Cell kv = result.get(0);
int compareResult = comparator.compareTo(kv.getValueArray(),
kv.getValueOffset(), kv.getValueLength());
switch (compareOp) {
case LESS:
@ -3490,14 +3491,14 @@ public class HRegion implements HeapSize { // , Writable{
}
@Override
public boolean next(List<KeyValue> outResults)
public boolean next(List<Cell> outResults)
throws IOException {
// apply the batching limit by default
return next(outResults, batch);
}
@Override
public synchronized boolean next(List<KeyValue> outResults, int limit) throws IOException {
public synchronized boolean next(List<Cell> outResults, int limit) throws IOException {
if (this.filterClosed) {
throw new UnknownScannerException("Scanner was closed (timed out?) " +
"after we renewed it. Could be caused by a very slow scanner " +
@ -3517,20 +3518,20 @@ public class HRegion implements HeapSize { // , Writable{
}
@Override
public boolean nextRaw(List<KeyValue> outResults)
public boolean nextRaw(List<Cell> outResults)
throws IOException {
return nextRaw(outResults, batch);
}
@Override
public boolean nextRaw(List<KeyValue> outResults, int limit) throws IOException {
public boolean nextRaw(List<Cell> outResults, int limit) throws IOException {
boolean returnResult;
if (outResults.isEmpty()) {
// Usually outResults is empty. This is true when next is called
// to handle scan or get operation.
returnResult = nextInternal(outResults, limit);
} else {
List<KeyValue> tmpList = new ArrayList<KeyValue>();
List<Cell> tmpList = new ArrayList<Cell>();
returnResult = nextInternal(tmpList, limit);
outResults.addAll(tmpList);
}
@ -3540,7 +3541,9 @@ public class HRegion implements HeapSize { // , Writable{
}
if (region != null && region.metricsRegion != null) {
long totalSize = 0;
for(KeyValue kv:outResults) {
for(Cell c:outResults) {
// TODO clean up
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
totalSize += kv.getLength();
}
region.metricsRegion.updateScanNext(totalSize);
@ -3549,7 +3552,7 @@ public class HRegion implements HeapSize { // , Writable{
}
private void populateFromJoinedHeap(List<KeyValue> results, int limit)
private void populateFromJoinedHeap(List<Cell> results, int limit)
throws IOException {
assert joinedContinuationRow != null;
KeyValue kv = populateResult(results, this.joinedHeap, limit,
@ -3574,7 +3577,7 @@ public class HRegion implements HeapSize { // , Writable{
* @param length length for currentRow
* @return KV_LIMIT if limit reached, next KeyValue otherwise.
*/
private KeyValue populateResult(List<KeyValue> results, KeyValueHeap heap, int limit,
private KeyValue populateResult(List<Cell> results, KeyValueHeap heap, int limit,
byte[] currentRow, int offset, short length) throws IOException {
KeyValue nextKv;
do {
@ -3596,7 +3599,7 @@ public class HRegion implements HeapSize { // , Writable{
return this.filter != null && this.filter.filterAllRemaining();
}
private boolean nextInternal(List<KeyValue> results, int limit)
private boolean nextInternal(List<Cell> results, int limit)
throws IOException {
if (!results.isEmpty()) {
throw new IllegalArgumentException("First parameter should be an empty list");
@ -3634,7 +3637,7 @@ public class HRegion implements HeapSize { // , Writable{
// First, check if we are at a stop row. If so, there are no more results.
if (stopRow) {
if (filter != null && filter.hasFilterRow()) {
filter.filterRow(results);
filter.filterRowCells(results);
}
return false;
}
@ -3667,7 +3670,7 @@ public class HRegion implements HeapSize { // , Writable{
// We have the part of the row necessary for filtering (all of it, usually).
// First filter with the filterRow(List).
if (filter != null && filter.hasFilterRow()) {
filter.filterRow(results);
filter.filterRowCells(results);
}
if (isEmptyRow) {
boolean moreRows = nextRow(currentRow, offset, length);
@ -4368,7 +4371,7 @@ public class HRegion implements HeapSize { // , Writable{
get.addFamily(family);
}
}
List<KeyValue> results = get(get, true);
List<Cell> results = get(get, true);
return new Result(results);
}
@ -4377,10 +4380,10 @@ public class HRegion implements HeapSize { // , Writable{
* @param withCoprocessor invoke coprocessor or not. We don't want to
* always invoke cp for this private method.
*/
private List<KeyValue> get(Get get, boolean withCoprocessor)
private List<Cell> get(Get get, boolean withCoprocessor)
throws IOException {
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> results = new ArrayList<Cell>();
// pre-get CP hook
if (withCoprocessor && (coprocessorHost != null)) {
@ -4409,8 +4412,8 @@ public class HRegion implements HeapSize { // , Writable{
if (this.metricsRegion != null) {
long totalSize = 0l;
if (results != null) {
for (KeyValue kv:results) {
totalSize += kv.getLength();
for (Cell kv:results) {
totalSize += KeyValueUtil.ensureKeyValue(kv).getLength();
}
}
this.metricsRegion.updateGet(totalSize);
@ -4687,7 +4690,7 @@ public class HRegion implements HeapSize { // , Writable{
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
get.addColumn(family.getKey(), kv.getQualifier());
}
List<KeyValue> results = get(get, false);
List<Cell> results = get(get, false);
// Iterate the input columns and update existing values if they were
// found, otherwise add new column initialized to the append value
@ -4700,9 +4703,8 @@ public class HRegion implements HeapSize { // , Writable{
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
KeyValue newKV;
if (idx < results.size()
&& results.get(idx).matchingQualifier(kv.getBuffer(),
kv.getQualifierOffset(), kv.getQualifierLength())) {
KeyValue oldKv = results.get(idx);
&& CellUtil.matchingQualifier(results.get(idx),kv)) {
KeyValue oldKv = KeyValueUtil.ensureKeyValue(results.get(idx));
// allocate an empty kv once
newKV = new KeyValue(row.length, kv.getFamilyLength(),
kv.getQualifierLength(), now, KeyValue.Type.Put,
@ -4862,19 +4864,17 @@ public class HRegion implements HeapSize { // , Writable{
get.addColumn(family.getKey(), kv.getQualifier());
}
get.setTimeRange(tr.getMin(), tr.getMax());
List<KeyValue> results = get(get, false);
List<Cell> results = get(get, false);
// Iterate the input columns and update existing values if they were
// found, otherwise add new column initialized to the increment amount
int idx = 0;
for (Cell cell: family.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
long amount = Bytes.toLong(kv.getValue());
byte [] qualifier = kv.getQualifier();
if (idx < results.size() && results.get(idx).matchingQualifier(qualifier)) {
kv = results.get(idx);
if(kv.getValueLength() == Bytes.SIZEOF_LONG) {
amount += Bytes.toLong(kv.getBuffer(), kv.getValueOffset(), Bytes.SIZEOF_LONG);
for (Cell kv: family.getValue()) {
long amount = Bytes.toLong(CellUtil.getValueArray(kv));
if (idx < results.size() && CellUtil.matchingQualifier(results.get(idx), kv)) {
Cell c = results.get(idx);
if(c.getValueLength() == Bytes.SIZEOF_LONG) {
amount += Bytes.toLong(c.getValueArray(), c.getValueOffset(), Bytes.SIZEOF_LONG);
} else {
// throw DoNotRetryIOException instead of IllegalArgumentException
throw new org.apache.hadoop.hbase.DoNotRetryIOException(
@ -4885,7 +4885,7 @@ public class HRegion implements HeapSize { // , Writable{
// Append new incremented KeyValue to list
KeyValue newKV =
new KeyValue(row, family.getKey(), qualifier, now, Bytes.toBytes(amount));
new KeyValue(row, family.getKey(), CellUtil.getQualifierArray(kv), now, Bytes.toBytes(amount));
newKV.setMvccVersion(w.getWriteNumber());
kvs.add(newKV);
@ -5143,7 +5143,7 @@ public class HRegion implements HeapSize { // , Writable{
// scan.addFamily(HConstants.CATALOG_FAMILY);
RegionScanner scanner = region.getScanner(scan);
try {
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
boolean done;
do {
kvs.clear();
@ -5454,15 +5454,15 @@ public class HRegion implements HeapSize { // , Writable{
/**
* A mocked list implementaion - discards all updates.
*/
private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
private static final List<Cell> MOCKED_LIST = new AbstractList<Cell>() {
@Override
public void add(int index, KeyValue element) {
public void add(int index, Cell element) {
// do nothing
}
@Override
public boolean addAll(int index, Collection<? extends KeyValue> c) {
public boolean addAll(int index, Collection<? extends Cell> c) {
return false; // this list is never changed as a result of an update
}

View File

@ -55,25 +55,26 @@ import org.apache.hadoop.classification.InterfaceAudience;
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.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.ClockOutOfSyncException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HealthCheckChore;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZNodeClearer;
@ -106,9 +107,9 @@ import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.RpcCallContext;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
import org.apache.hadoop.hbase.master.SplitLogManager;
@ -3069,8 +3070,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
if (!results.isEmpty()) {
for (Result r : results) {
if (maxScannerResultSize < Long.MAX_VALUE){
for (KeyValue kv : r.raw()) {
currentScanResultSize += kv.heapSize();
for (Cell kv : r.raw()) {
// TODO
currentScanResultSize += KeyValueUtil.ensureKeyValue(kv).heapSize();
}
}
}
@ -3085,7 +3087,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
if (maxResultSize <= 0) {
maxResultSize = maxScannerResultSize;
}
List<KeyValue> values = new ArrayList<KeyValue>();
List<Cell> values = new ArrayList<Cell>();
MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
region.startRegionOperation(Operation.SCAN);
try {
@ -3097,8 +3099,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
boolean moreRows = scanner.nextRaw(values);
if (!values.isEmpty()) {
if (maxScannerResultSize < Long.MAX_VALUE){
for (KeyValue kv : values) {
currentScanResultSize += kv.heapSize();
for (Cell kv : values) {
currentScanResultSize += KeyValueUtil.ensureKeyValue(kv).heapSize();
}
}
results.add(new Result(values));

View File

@ -18,13 +18,13 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
/**
* Internal scanners differ from client-side scanners in that they operate on
* HStoreKeys and byte[] instead of RowResults. This is because they are
@ -47,7 +47,7 @@ public interface InternalScanner extends Closeable {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
boolean next(List<KeyValue> results) throws IOException;
boolean next(List<Cell> results) throws IOException;
/**
* Grab the next row's worth of values with a limit on the number of values
@ -57,7 +57,7 @@ public interface InternalScanner extends Closeable {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
boolean next(List<KeyValue> result, int limit) throws IOException;
boolean next(List<Cell> result, int limit) throws IOException;
/**
* Closes the scanner and releases any resources it has allocated

View File

@ -25,6 +25,7 @@ import java.util.List;
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;
@ -120,7 +121,7 @@ public class KeyValueHeap extends NonLazyKeyValueScanner
* @param limit
* @return true if there are more keys, false if all scanners are done
*/
public boolean next(List<KeyValue> result, int limit) throws IOException {
public boolean next(List<Cell> result, int limit) throws IOException {
if (this.current == null) {
return false;
}
@ -153,7 +154,7 @@ public class KeyValueHeap extends NonLazyKeyValueScanner
* @param result
* @return true if there are more keys, false if all scanners are done
*/
public boolean next(List<KeyValue> result) throws IOException {
public boolean next(List<Cell> result) throws IOException {
return next(result, -1);
}

View File

@ -34,21 +34,21 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@ -786,7 +786,7 @@ public class RegionCoprocessorHost
* @return true if default processing should be bypassed
* @exception IOException Exception
*/
public boolean preGet(final Get get, final List<KeyValue> results)
public boolean preGet(final Get get, final List<Cell> results)
throws IOException {
boolean bypass = false;
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
@ -794,7 +794,7 @@ public class RegionCoprocessorHost
if (env.getInstance() instanceof RegionObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((RegionObserver)env.getInstance()).preGet(ctx, get, results);
((RegionObserver)env.getInstance()).preGetOp(ctx, get, results);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}
@ -809,17 +809,17 @@ public class RegionCoprocessorHost
/**
* @param get the Get request
* @param results the result set
* @param results the result sett
* @exception IOException Exception
*/
public void postGet(final Get get, final List<KeyValue> results)
public void postGet(final Get get, final List<Cell> results)
throws IOException {
ObserverContext<RegionCoprocessorEnvironment> ctx = null;
for (RegionEnvironment env: coprocessors) {
if (env.getInstance() instanceof RegionObserver) {
ctx = ObserverContext.createAndPrepare(env, ctx);
try {
((RegionObserver)env.getInstance()).postGet(ctx, get, results);
((RegionObserver)env.getInstance()).postGetOp(ctx, get, results);
} catch (Throwable e) {
handleCoprocessorThrowable(env, e);
}

View File

@ -22,8 +22,8 @@ import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
/**
@ -74,7 +74,7 @@ public interface RegionScanner extends InternalScanner {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
boolean nextRaw(List<KeyValue> result) throws IOException;
boolean nextRaw(List<Cell> result) throws IOException;
/**
* Grab the next row's worth of values with a limit on the number of values
@ -102,5 +102,5 @@ public interface RegionScanner extends InternalScanner {
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
*/
boolean nextRaw(List<KeyValue> result, int limit) throws IOException;
boolean nextRaw(List<Cell> result, int limit) throws IOException;
}

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import java.util.NavigableSet;
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.client.Scan;
@ -303,7 +304,7 @@ public class ScanQueryMatcher {
* 7. Delete marker need to be version counted together with puts
* they affect
*/
byte type = kv.getType();
byte type = kv.getTypeByte();
if (kv.isDelete()) {
if (!keepDeletedCells) {
// first ignore delete markers if the scanner can do so, and the
@ -480,11 +481,11 @@ public class ScanQueryMatcher {
return this.filter;
}
public KeyValue getNextKeyHint(KeyValue kv) throws IOException {
public Cell getNextKeyHint(Cell kv) throws IOException {
if (filter == null) {
return null;
} else {
return filter.getNextKeyHint(kv);
return filter.getNextCellHint(kv);
}
}

View File

@ -30,8 +30,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
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.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
@ -117,16 +119,17 @@ s */
Compactor.CellSink sink, long smallestReadPoint) throws IOException {
int compactionKVMax =
conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
boolean hasMore;
long flushed = 0;
do {
hasMore = scanner.next(kvs, compactionKVMax);
if (!kvs.isEmpty()) {
for (KeyValue kv : kvs) {
for (Cell c : kvs) {
// If we know that this KV is going to be included always, then let us
// set its memstoreTS to 0. This will help us save space when writing to
// disk.
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
if (kv.getMvccVersion() <= smallestReadPoint) {
// let us not change the original KV. It could be in the memstore
// changing its memstoreTS could affect other threads/scanners.

View File

@ -29,9 +29,11 @@ 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.Cell;
import org.apache.hadoop.hbase.DoNotRetryIOException;
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.executor.ExecutorService;
import org.apache.hadoop.hbase.filter.Filter;
@ -358,7 +360,7 @@ public class StoreScanner extends NonLazyKeyValueScanner
* @return true if there are more rows, false if scanner is done
*/
@Override
public synchronized boolean next(List<KeyValue> outResult, int limit) throws IOException {
public synchronized boolean next(List<Cell> outResult, int limit) throws IOException {
if (checkReseek()) {
return true;
}
@ -410,7 +412,8 @@ public class StoreScanner extends NonLazyKeyValueScanner
Filter f = matcher.getFilter();
if (f != null) {
kv = f.transform(kv);
// TODO convert Scan Query Matcher to be Cell instead of KV based ?
kv = KeyValueUtil.ensureKeyValue(f.transformCell(kv));
}
this.countPerRow++;
@ -473,7 +476,8 @@ public class StoreScanner extends NonLazyKeyValueScanner
break;
case SEEK_NEXT_USING_HINT:
KeyValue nextKV = matcher.getNextKeyHint(kv);
// TODO convert resee to Cell?
KeyValue nextKV = KeyValueUtil.ensureKeyValue(matcher.getNextKeyHint(kv));
if (nextKV != null) {
reseek(nextKV);
} else {
@ -496,7 +500,7 @@ public class StoreScanner extends NonLazyKeyValueScanner
}
@Override
public synchronized boolean next(List<KeyValue> outResult) throws IOException {
public synchronized boolean next(List<Cell> outResult) throws IOException {
return next(outResult, -1);
}

View File

@ -28,8 +28,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
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.io.CellOutputStream;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -198,14 +200,15 @@ public abstract class Compactor {
int bytesWritten = 0;
// Since scanner.next() can return 'false' but still be delivering data,
// we have to use a do/while loop.
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
// Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
int closeCheckInterval = HStore.getCloseCheckInterval();
boolean hasMore;
do {
hasMore = scanner.next(kvs, compactionKVMax);
// output to writer:
for (KeyValue kv : kvs) {
for (Cell c : kvs) {
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
if (kv.getMvccVersion() <= smallestReadPoint) {
kv.setMvccVersion(0);
}

View File

@ -18,14 +18,7 @@
*/
package org.apache.hadoop.hbase.rest;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.rest.ResourceBase;
import org.apache.hadoop.hbase.rest.RowSpec;
import org.apache.hadoop.hbase.rest.TableResource;
import org.apache.hadoop.hbase.rest.model.CellModel;
import org.apache.hadoop.hbase.rest.model.CellSetModel;
import org.apache.hadoop.hbase.rest.model.RowModel;
import java.io.IOException;
import javax.ws.rs.GET;
import javax.ws.rs.Produces;
@ -33,7 +26,13 @@ import javax.ws.rs.core.Context;
import javax.ws.rs.core.MultivaluedMap;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriInfo;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.rest.model.CellModel;
import org.apache.hadoop.hbase.rest.model.CellSetModel;
import org.apache.hadoop.hbase.rest.model.RowModel;
@InterfaceAudience.Private
public class MultiRowResource extends ResourceBase {
@ -83,12 +82,13 @@ public class MultiRowResource extends ResourceBase {
.build();
}
KeyValue value = null;
Cell value = null;
RowModel rowModel = new RowModel(rk);
while ((value = generator.next()) != null) {
rowModel.addCell(new CellModel(value.getFamily(), value.getQualifier(),
value.getTimestamp(), value.getValue()));
rowModel.addCell(new CellModel(CellUtil.getFamilyArray(value),
CellUtil.getQualifierArray(value),
value.getTimestamp(), CellUtil.getValueArray(value)));
}
model.addRow(rowModel);

View File

@ -21,14 +21,14 @@ package org.apache.hadoop.hbase.rest;
import java.io.IOException;
import java.util.Iterator;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.rest.model.ScannerModel;
@InterfaceAudience.Private
public abstract class ResultGenerator implements Iterator<KeyValue> {
public abstract class ResultGenerator implements Iterator<Cell> {
public static ResultGenerator fromRowSpec(final String table,
final RowSpec rowspec, final Filter filter) throws IOException {
@ -43,7 +43,7 @@ public abstract class ResultGenerator implements Iterator<KeyValue> {
return ScannerModel.buildFilter(filter);
}
public abstract void putBack(KeyValue kv);
public abstract void putBack(Cell kv);
public abstract void close();
}

View File

@ -38,6 +38,8 @@ import javax.ws.rs.core.UriInfo;
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.TableNotFoundException;
@ -96,17 +98,17 @@ public class RowResource extends ResourceBase {
}
int count = 0;
CellSetModel model = new CellSetModel();
KeyValue value = generator.next();
byte[] rowKey = value.getRow();
Cell value = generator.next();
byte[] rowKey = CellUtil.getRowArray(value);
RowModel rowModel = new RowModel(rowKey);
do {
if (!Bytes.equals(value.getRow(), rowKey)) {
if (!Bytes.equals(CellUtil.getRowArray(value), rowKey)) {
model.addRow(rowModel);
rowKey = value.getRow();
rowKey = CellUtil.getRowArray(value);
rowModel = new RowModel(rowKey);
}
rowModel.addCell(new CellModel(value.getFamily(), value.getQualifier(),
value.getTimestamp(), value.getValue()));
rowModel.addCell(new CellModel(CellUtil.getFamilyArray(value), CellUtil.getQualifierArray(value),
value.getTimestamp(), CellUtil.getValueArray(value)));
if (++count > rowspec.getMaxValues()) {
break;
}
@ -155,8 +157,8 @@ public class RowResource extends ResourceBase {
.type(MIMETYPE_TEXT).entity("Not found" + CRLF)
.build();
}
KeyValue value = generator.next();
ResponseBuilder response = Response.ok(value.getValue());
Cell value = generator.next();
ResponseBuilder response = Response.ok(CellUtil.getValueArray(value));
response.header("X-Timestamp", value.getTimestamp());
servlet.getMetrics().incrementSucessfulGetRequests(1);
return response.build();

View File

@ -26,6 +26,7 @@ import java.util.NoSuchElementException;
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.DoNotRetryIOException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get;
@ -38,8 +39,8 @@ import org.apache.hadoop.util.StringUtils;
public class RowResultGenerator extends ResultGenerator {
private static final Log LOG = LogFactory.getLog(RowResultGenerator.class);
private Iterator<KeyValue> valuesI;
private KeyValue cache;
private Iterator<Cell> valuesI;
private Cell cache;
public RowResultGenerator(final String tableName, final RowSpec rowspec,
final Filter filter) throws IllegalArgumentException, IOException {
@ -91,9 +92,9 @@ public class RowResultGenerator extends ResultGenerator {
return valuesI.hasNext();
}
public KeyValue next() {
public Cell next() {
if (cache != null) {
KeyValue kv = cache;
Cell kv = cache;
cache = null;
return kv;
}
@ -107,7 +108,7 @@ public class RowResultGenerator extends ResultGenerator {
}
}
public void putBack(KeyValue kv) {
public void putBack(Cell kv) {
this.cache = kv;
}

View File

@ -35,6 +35,8 @@ 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.rest.model.CellModel;
import org.apache.hadoop.hbase.rest.model.CellSetModel;
@ -91,7 +93,7 @@ public class ScannerInstanceResource extends ResourceBase {
}
int count = limit;
do {
KeyValue value = null;
Cell value = null;
try {
value = generator.next();
} catch (IllegalStateException e) {
@ -115,10 +117,10 @@ public class ScannerInstanceResource extends ResourceBase {
break;
}
if (rowKey == null) {
rowKey = value.getRow();
rowKey = CellUtil.getRowArray(value);
rowModel = new RowModel(rowKey);
}
if (!Bytes.equals(value.getRow(), rowKey)) {
if (!Bytes.equals(CellUtil.getRowArray(value), rowKey)) {
// if maxRows was given as a query param, stop if we would exceed the
// specified number of rows
if (maxRows > 0) {
@ -128,12 +130,12 @@ public class ScannerInstanceResource extends ResourceBase {
}
}
model.addRow(rowModel);
rowKey = value.getRow();
rowKey = CellUtil.getRowArray(value);
rowModel = new RowModel(rowKey);
}
rowModel.addCell(
new CellModel(value.getFamily(), value.getQualifier(),
value.getTimestamp(), value.getValue()));
new CellModel(CellUtil.getFamilyArray(value), CellUtil.getQualifierArray(value),
value.getTimestamp(), CellUtil.getValueArray(value)));
} while (--count > 0);
model.addRow(rowModel);
ResponseBuilder response = Response.ok(model);
@ -151,17 +153,17 @@ public class ScannerInstanceResource extends ResourceBase {
}
servlet.getMetrics().incrementRequests(1);
try {
KeyValue value = generator.next();
Cell value = generator.next();
if (value == null) {
LOG.info("generator exhausted");
return Response.noContent().build();
}
ResponseBuilder response = Response.ok(value.getValue());
ResponseBuilder response = Response.ok(CellUtil.getValueArray(value));
response.cacheControl(cacheControl);
response.header("X-Row", Base64.encodeBytes(value.getRow()));
response.header("X-Row", Base64.encodeBytes(CellUtil.getRowArray(value)));
response.header("X-Column",
Base64.encodeBytes(
KeyValue.makeColumn(value.getFamily(), value.getQualifier())));
KeyValue.makeColumn(CellUtil.getFamilyArray(value), CellUtil.getQualifierArray(value))));
response.header("X-Timestamp", value.getTimestamp());
servlet.getMetrics().incrementSucessfulGetRequests(1);
return response.build();

View File

@ -25,6 +25,7 @@ import java.util.Iterator;
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.KeyValue;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.HTableInterface;
@ -51,8 +52,8 @@ public class ScannerResultGenerator extends ResultGenerator {
}
private String id;
private Iterator<KeyValue> rowI;
private KeyValue cache;
private Iterator<Cell> rowI;
private Cell cache;
private ResultScanner scanner;
private Result cached;
@ -131,9 +132,9 @@ public class ScannerResultGenerator extends ResultGenerator {
return cached != null;
}
public KeyValue next() {
public Cell next() {
if (cache != null) {
KeyValue kv = cache;
Cell kv = cache;
cache = null;
return kv;
}
@ -169,7 +170,7 @@ public class ScannerResultGenerator extends ResultGenerator {
return null;
}
public void putBack(KeyValue kv) {
public void putBack(Cell kv) {
this.cache = kv;
}

View File

@ -168,7 +168,7 @@ public class RemoteHTable implements HTableInterface {
protected Result[] buildResultFromModel(final CellSetModel model) {
List<Result> results = new ArrayList<Result>();
for (RowModel row: model.getRows()) {
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
for (CellModel cell: row.getCells()) {
byte[][] split = KeyValue.parseColumn(cell.getColumn());
byte[] column = split[0];

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.security.access;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.KeyValue;
@ -59,7 +61,9 @@ class AccessControlFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
public ReturnCode filterKeyValue(Cell c) {
// TODO go and redo auth manager to use Cell instead of KV.
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
if (authManager.authorize(user, table, kv, TablePermission.Action.READ)) {
return ReturnCode.INCLUDE;
}

View File

@ -33,13 +33,14 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
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.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -48,6 +49,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.hbase.filter.RegexStringComparator;
@ -367,14 +369,14 @@ public class AccessControlLists {
iScanner = aclRegion.getScanner(scan);
while (true) {
List<KeyValue> row = new ArrayList<KeyValue>();
List<Cell> row = new ArrayList<Cell>();
boolean hasNext = iScanner.next(row);
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
byte[] entry = null;
for (KeyValue kv : row) {
for (Cell kv : row) {
if (entry == null) {
entry = kv.getRow();
entry = CellUtil.getRowArray(kv);
}
Pair<String,TablePermission> permissionsOfUserOnTable =
parsePermissionRecord(entry, kv);
@ -511,7 +513,7 @@ public class AccessControlLists {
byte[] entryName, Result result) {
ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
if (result != null && result.size() > 0) {
for (KeyValue kv : result.raw()) {
for (Cell kv : result.raw()) {
Pair<String,TablePermission> permissionsOfUserOnTable =
parsePermissionRecord(entryName, kv);
@ -527,16 +529,16 @@ public class AccessControlLists {
}
private static Pair<String, TablePermission> parsePermissionRecord(
byte[] entryName, KeyValue kv) {
byte[] entryName, Cell kv) {
// return X given a set of permissions encoded in the permissionRecord kv.
byte[] family = kv.getFamily();
byte[] family = CellUtil.getFamilyArray(kv);
if (!Bytes.equals(family, ACL_LIST_FAMILY)) {
return null;
}
byte[] key = kv.getQualifier();
byte[] value = kv.getValue();
byte[] key = CellUtil.getQualifierArray(kv);
byte[] value = CellUtil.getValueArray(kv);
if (LOG.isDebugEnabled()) {
LOG.debug("Read acl: kv ["+
Bytes.toStringBinary(key)+": "+

View File

@ -944,8 +944,8 @@ public class AccessController extends BaseRegionObserver
}
@Override
public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c,
final Get get, final List<KeyValue> result) throws IOException {
public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c,
final Get get, final List<Cell> result) throws IOException {
/*
if column family level checks fail, check for a qualifier level permission
in one of the families. If it is present, then continue with the AccessControlFilter.

View File

@ -24,6 +24,8 @@ import java.util.List;
import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Increment;
@ -33,8 +35,8 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TIncrement;
import org.apache.hadoop.hbase.thrift.generated.TColumn;
import org.apache.hadoop.hbase.thrift.generated.TIncrement;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
@ -98,10 +100,10 @@ public class ThriftUtilities {
* Hbase Cell object
* @return Thrift TCell array
*/
static public List<TCell> cellFromHBase(KeyValue in) {
static public List<TCell> cellFromHBase(Cell in) {
List<TCell> list = new ArrayList<TCell>(1);
if (in != null) {
list.add(new TCell(ByteBuffer.wrap(in.getValue()), in.getTimestamp()));
list.add(new TCell(ByteBuffer.wrap(CellUtil.getValueArray(in)), in.getTimestamp()));
}
return list;
}
@ -112,12 +114,12 @@ public class ThriftUtilities {
* @param in Hbase Cell array
* @return Thrift TCell array
*/
static public List<TCell> cellFromHBase(KeyValue[] in) {
static public List<TCell> cellFromHBase(Cell[] in) {
List<TCell> list = null;
if (in != null) {
list = new ArrayList<TCell>(in.length);
for (int i = 0; i < in.length; i++) {
list.add(new TCell(ByteBuffer.wrap(in[i].getValue()), in[i].getTimestamp()));
list.add(new TCell(ByteBuffer.wrap(CellUtil.getValueArray(in[i])), in[i].getTimestamp()));
}
} else {
list = new ArrayList<TCell>(0);
@ -150,19 +152,19 @@ public class ThriftUtilities {
result.row = ByteBuffer.wrap(result_.getRow());
if (sortColumns) {
result.sortedColumns = new ArrayList<TColumn>();
for (KeyValue kv : result_.raw()) {
for (Cell kv : result_.raw()) {
result.sortedColumns.add(new TColumn(
ByteBuffer.wrap(KeyValue.makeColumn(kv.getFamily(),
kv.getQualifier())),
new TCell(ByteBuffer.wrap(kv.getValue()), kv.getTimestamp())));
ByteBuffer.wrap(KeyValue.makeColumn(CellUtil.getFamilyArray(kv),
CellUtil.getQualifierArray(kv))),
new TCell(ByteBuffer.wrap(CellUtil.getValueArray(kv)), kv.getTimestamp())));
}
} else {
result.columns = new TreeMap<ByteBuffer, TCell>();
for (KeyValue kv : result_.raw()) {
for (Cell kv : result_.raw()) {
result.columns.put(
ByteBuffer.wrap(KeyValue.makeColumn(kv.getFamily(),
kv.getQualifier())),
new TCell(ByteBuffer.wrap(kv.getValue()), kv.getTimestamp()));
ByteBuffer.wrap(KeyValue.makeColumn(CellUtil.getFamilyArray(kv),
CellUtil.getQualifierArray(kv))),
new TCell(ByteBuffer.wrap(CellUtil.getValueArray(kv)), kv.getTimestamp()));
}
}
results.add(result);

View File

@ -18,20 +18,45 @@
*/
package org.apache.hadoop.hbase.thrift2;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.*;
import org.apache.hadoop.hbase.filter.ParseFilter;
import org.apache.hadoop.hbase.thrift2.generated.*;
import org.apache.hadoop.hbase.util.Bytes;
import static org.apache.hadoop.hbase.util.Bytes.getBytes;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import static org.apache.hadoop.hbase.util.Bytes.getBytes;
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.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.OperationWithAttributes;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.ParseFilter;
import org.apache.hadoop.hbase.thrift2.generated.TColumn;
import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
import org.apache.hadoop.hbase.thrift2.generated.TDelete;
import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
import org.apache.hadoop.hbase.thrift2.generated.TDurability;
import org.apache.hadoop.hbase.thrift2.generated.TGet;
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
import org.apache.hadoop.hbase.thrift2.generated.TMutation;
import org.apache.hadoop.hbase.thrift2.generated.TPut;
import org.apache.hadoop.hbase.thrift2.generated.TResult;
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
import org.apache.hadoop.hbase.thrift2.generated.TScan;
import org.apache.hadoop.hbase.thrift2.generated.TTimeRange;
import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private
public class ThriftUtilities {
@ -115,19 +140,19 @@ public class ThriftUtilities {
* @return converted result, returns an empty result if the input is <code>null</code>
*/
public static TResult resultFromHBase(Result in) {
KeyValue[] raw = in.raw();
Cell[] raw = in.raw();
TResult out = new TResult();
byte[] row = in.getRow();
if (row != null) {
out.setRow(in.getRow());
}
List<TColumnValue> columnValues = new ArrayList<TColumnValue>();
for (KeyValue kv : raw) {
for (Cell kv : raw) {
TColumnValue col = new TColumnValue();
col.setFamily(kv.getFamily());
col.setQualifier(kv.getQualifier());
col.setFamily(CellUtil.getFamilyArray(kv));
col.setQualifier(CellUtil.getQualifierArray(kv));
col.setTimestamp(kv.getTimestamp());
col.setValue(kv.getValue());
col.setValue(CellUtil.getValueArray(kv));
columnValues.add(col);
}
out.setColumnValues(columnValues);

View File

@ -56,8 +56,8 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
@ -2559,8 +2560,8 @@ public class HBaseFsck extends Configured implements Tool {
int countRecord = 1;
// comparator to sort KeyValues with latest modtime
final Comparator<KeyValue> comp = new Comparator<KeyValue>() {
public int compare(KeyValue k1, KeyValue k2) {
final Comparator<Cell> comp = new Comparator<Cell>() {
public int compare(Cell k1, Cell k2) {
return (int)(k1.getTimestamp() - k2.getTimestamp());
}
};

View File

@ -40,7 +40,8 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.*;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hdfs.MiniDFSCluster;
/**
@ -514,7 +515,7 @@ public abstract class HBaseTestCase extends TestCase {
public interface ScannerIncommon
extends Iterable<Result> {
boolean next(List<KeyValue> values)
boolean next(List<Cell> values)
throws IOException;
void close() throws IOException;
@ -526,7 +527,8 @@ public abstract class HBaseTestCase extends TestCase {
this.scanner = scanner;
}
public boolean next(List<KeyValue> values)
@Override
public boolean next(List<Cell> values)
throws IOException {
Result results = scanner.next();
if (results == null) {
@ -542,7 +544,7 @@ public abstract class HBaseTestCase extends TestCase {
}
@SuppressWarnings("unchecked")
public Iterator iterator() {
public Iterator<Result> iterator() {
return scanner.iterator();
}
}
@ -554,15 +556,18 @@ public abstract class HBaseTestCase extends TestCase {
this.scanner = scanner;
}
public boolean next(List<KeyValue> results)
@Override
public boolean next(List<Cell> results)
throws IOException {
return scanner.next(results);
}
@Override
public void close() throws IOException {
scanner.close();
}
@Override
public Iterator<Result> iterator() {
throw new UnsupportedOperationException();
}

View File

@ -2674,19 +2674,19 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Do a small get/scan against one store. This is required because store
* has no actual methods of querying itself, and relies on StoreScanner.
*/
public static List<KeyValue> getFromStoreFile(HStore store,
public static List<Cell> getFromStoreFile(HStore store,
Get get) throws IOException {
MultiVersionConsistencyControl.resetThreadReadPoint();
Scan scan = new Scan(get);
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
scan.getFamilyMap().get(store.getFamily().getName()));
List<KeyValue> result = new ArrayList<KeyValue>();
List<Cell> result = new ArrayList<Cell>();
scanner.next(result);
if (!result.isEmpty()) {
// verify that we are on the row we want:
KeyValue kv = result.get(0);
if (!Bytes.equals(kv.getRow(), get.getRow())) {
Cell kv = result.get(0);
if (!CellUtil.matchingRow(kv, get.getRow())) {
result.clear();
}
}
@ -2717,7 +2717,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* Do a small get/scan against one store. This is required because store
* has no actual methods of querying itself, and relies on StoreScanner.
*/
public static List<KeyValue> getFromStoreFile(HStore store,
public static List<Cell> getFromStoreFile(HStore store,
byte [] row,
NavigableSet<byte[]> columns
) throws IOException {
@ -2778,8 +2778,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
public static void assertKVListsEqual(String additionalMsg,
final List<KeyValue> expected,
final List<KeyValue> actual) {
final List<? extends Cell> expected,
final List<? extends Cell> actual) {
final int eLen = expected.size();
final int aLen = actual.size();
final int minLen = Math.min(eLen, aLen);

View File

@ -173,10 +173,10 @@ public class TestAcidGuarantees implements Tool {
msg.append("Failed after ").append(numVerified).append("!");
msg.append("Expected=").append(Bytes.toStringBinary(expected));
msg.append("Got:\n");
for (KeyValue kv : res.list()) {
for (Cell kv : res.list()) {
msg.append(kv.toString());
msg.append(" val= ");
msg.append(Bytes.toStringBinary(kv.getValue()));
msg.append(Bytes.toStringBinary(CellUtil.getValueArray(kv)));
msg.append("\n");
}
throw new RuntimeException(msg.toString());
@ -230,10 +230,10 @@ public class TestAcidGuarantees implements Tool {
msg.append("Failed after ").append(numRowsScanned).append("!");
msg.append("Expected=").append(Bytes.toStringBinary(expected));
msg.append("Got:\n");
for (KeyValue kv : res.list()) {
for (Cell kv : res.list()) {
msg.append(kv.toString());
msg.append(" val= ");
msg.append(Bytes.toStringBinary(kv.getValue()));
msg.append(Bytes.toStringBinary(CellUtil.getValueArray(kv)));
msg.append("\n");
}
throw new RuntimeException(msg.toString());

View File

@ -236,7 +236,7 @@ public class TestMultiVersions {
get.setTimeStamp(timestamp[j]);
Result result = table.get(get);
int cellCount = 0;
for(@SuppressWarnings("unused")KeyValue kv : result.list()) {
for(@SuppressWarnings("unused")Cell kv : result.list()) {
cellCount++;
}
assertTrue(cellCount == 1);

View File

@ -107,7 +107,7 @@ public class TimestampTestBase extends HBaseTestCase {
get.setMaxVersions(3);
Result result = incommon.get(get);
assertEquals(1, result.size());
long time = Bytes.toLong(result.raw()[0].getValue());
long time = Bytes.toLong(CellUtil.getValueArray(result.raw()[0]));
assertEquals(time, currentTime);
}
@ -136,10 +136,10 @@ public class TimestampTestBase extends HBaseTestCase {
get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
get.setMaxVersions(tss.length);
Result result = incommon.get(get);
KeyValue [] kvs = result.raw();
Cell [] kvs = result.raw();
assertEquals(kvs.length, tss.length);
for(int i=0;i<kvs.length;i++) {
t = Bytes.toLong(kvs[i].getValue());
t = Bytes.toLong(CellUtil.getValueArray(kvs[i]));
assertEquals(tss[i], t);
}
@ -155,7 +155,7 @@ public class TimestampTestBase extends HBaseTestCase {
kvs = result.raw();
assertEquals(kvs.length, tss.length - 1);
for(int i=1;i<kvs.length;i++) {
t = Bytes.toLong(kvs[i-1].getValue());
t = Bytes.toLong(CellUtil.getValueArray(kvs[i-1]));
assertEquals(tss[i], t);
}

View File

@ -23,6 +23,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
@ -72,7 +73,7 @@ public class MetaMockingUtil {
*/
public static Result getMetaTableRowResult(HRegionInfo region, final ServerName sn,
HRegionInfo splita, HRegionInfo splitb) throws IOException {
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
if (region != null) {
kvs.add(new KeyValue(
region.getRegionName(),

View File

@ -30,6 +30,7 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.TableName;
@ -97,7 +98,7 @@ public class TestMetaReaderEditorNoCluster {
public void testGetHRegionInfo() throws IOException {
assertNull(HRegionInfo.getHRegionInfo(new Result()));
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
Result r = new Result(kvs);
assertNull(HRegionInfo.getHRegionInfo(r));
@ -152,7 +153,7 @@ public class TestMetaReaderEditorNoCluster {
// show. We will know if they happened or not because we will ask
// mockito at the end of this test to verify that scan was indeed
// called the wanted number of times.
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,

View File

@ -48,6 +48,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
@ -205,9 +207,9 @@ public class TestFromClientSide {
s.setTimeRange(0, ts+3);
s.setMaxVersions();
ResultScanner scanner = h.getScanner(s);
KeyValue[] kvs = scanner.next().raw();
assertArrayEquals(T2, kvs[0].getValue());
assertArrayEquals(T1, kvs[1].getValue());
Cell[] kvs = scanner.next().raw();
assertArrayEquals(T2, CellUtil.getValueArray(kvs[0]));
assertArrayEquals(T1, CellUtil.getValueArray(kvs[1]));
scanner.close();
s = new Scan(T1);
@ -215,11 +217,11 @@ public class TestFromClientSide {
s.setMaxVersions();
scanner = h.getScanner(s);
kvs = scanner.next().raw();
assertTrue(kvs[0].isDeleteFamily());
assertArrayEquals(T3, kvs[1].getValue());
assertTrue(kvs[2].isDelete());
assertArrayEquals(T2, kvs[3].getValue());
assertArrayEquals(T1, kvs[4].getValue());
assertTrue(CellUtil.isDeleteFamily(kvs[0]));
assertArrayEquals(T3, CellUtil.getValueArray(kvs[1]));
assertTrue(CellUtil.isDelete(kvs[2]));
assertArrayEquals(T2, CellUtil.getValueArray(kvs[3]));
assertArrayEquals(T1, CellUtil.getValueArray(kvs[4]));
scanner.close();
h.close();
}
@ -475,9 +477,9 @@ public class TestFromClientSide {
while (scanner.hasNext()) {
Result result = scanner.next();
System.out.println("Got back key: " + Bytes.toString(result.getRow()));
for (KeyValue kv : result.raw()) {
for (Cell kv : result.raw()) {
System.out.println("kv=" + kv.toString() + ", "
+ Bytes.toString(kv.getValue()));
+ Bytes.toString(CellUtil.getValueArray(kv)));
}
numberOfResults++;
}
@ -746,8 +748,8 @@ public class TestFromClientSide {
int expectedIndex = 1;
for(Result result : ht.getScanner(scan)) {
assertEquals(result.size(), 1);
assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[expectedIndex]));
assertTrue(Bytes.equals(result.raw()[0].getQualifier(),
assertTrue(Bytes.equals(CellUtil.getRowArray(result.raw()[0]), ROWS[expectedIndex]));
assertTrue(Bytes.equals(CellUtil.getQualifierArray(result.raw()[0]),
QUALIFIERS[expectedIndex]));
expectedIndex++;
}
@ -782,7 +784,7 @@ public class TestFromClientSide {
for(Result result : ht.getScanner(scan)) {
assertEquals(result.size(), 1);
assertEquals(result.raw()[0].getValueLength(), Bytes.SIZEOF_INT);
assertEquals(Bytes.toInt(result.raw()[0].getValue()), VALUE.length);
assertEquals(Bytes.toInt(CellUtil.getValueArray(result.raw()[0])), VALUE.length);
count++;
}
assertEquals(count, 10);
@ -2133,15 +2135,15 @@ public class TestFromClientSide {
result = scanner.next();
assertTrue("Expected 1 key but received " + result.size(),
result.size() == 1);
assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[3]));
assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[0]));
assertTrue(Bytes.equals(CellUtil.getRowArray(result.raw()[0]), ROWS[3]));
assertTrue(Bytes.equals(CellUtil.getValueArray(result.raw()[0]), VALUES[0]));
result = scanner.next();
assertTrue("Expected 2 keys but received " + result.size(),
result.size() == 2);
assertTrue(Bytes.equals(result.raw()[0].getRow(), ROWS[4]));
assertTrue(Bytes.equals(result.raw()[1].getRow(), ROWS[4]));
assertTrue(Bytes.equals(result.raw()[0].getValue(), VALUES[1]));
assertTrue(Bytes.equals(result.raw()[1].getValue(), VALUES[2]));
assertTrue(Bytes.equals(CellUtil.getRowArray(result.raw()[0]), ROWS[4]));
assertTrue(Bytes.equals(CellUtil.getRowArray(result.raw()[1]), ROWS[4]));
assertTrue(Bytes.equals(CellUtil.getValueArray(result.raw()[0]), VALUES[1]));
assertTrue(Bytes.equals(CellUtil.getValueArray(result.raw()[1]), VALUES[2]));
scanner.close();
// Add test of bulk deleting.
@ -2269,7 +2271,7 @@ public class TestFromClientSide {
Get get = new Get(ROWS[numRows-1]);
Result result = ht.get(get);
assertNumKeys(result, numColsPerRow);
KeyValue [] keys = result.raw();
Cell [] keys = result.raw();
for(int i=0;i<result.size();i++) {
assertKey(keys[i], ROWS[numRows-1], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -2280,7 +2282,7 @@ public class TestFromClientSide {
int rowCount = 0;
while((result = scanner.next()) != null) {
assertNumKeys(result, numColsPerRow);
KeyValue [] kvs = result.raw();
Cell [] kvs = result.raw();
for(int i=0;i<numColsPerRow;i++) {
assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -2309,7 +2311,7 @@ public class TestFromClientSide {
rowCount = 0;
while((result = scanner.next()) != null) {
assertNumKeys(result, numColsPerRow);
KeyValue [] kvs = result.raw();
Cell [] kvs = result.raw();
for(int i=0;i<numColsPerRow;i++) {
assertKey(kvs[i], ROWS[rowCount], FAMILY, QUALIFIERS[i], QUALIFIERS[i]);
}
@ -3082,38 +3084,38 @@ public class TestFromClientSide {
// Verifiers
//
private void assertKey(KeyValue key, byte [] row, byte [] family,
private void assertKey(Cell key, byte [] row, byte [] family,
byte [] qualifier, byte [] value)
throws Exception {
assertTrue("Expected row [" + Bytes.toString(row) + "] " +
"Got row [" + Bytes.toString(key.getRow()) +"]",
equals(row, key.getRow()));
"Got row [" + Bytes.toString(CellUtil.getRowArray(key)) +"]",
equals(row, CellUtil.getRowArray(key)));
assertTrue("Expected family [" + Bytes.toString(family) + "] " +
"Got family [" + Bytes.toString(key.getFamily()) + "]",
equals(family, key.getFamily()));
"Got family [" + Bytes.toString(CellUtil.getFamilyArray(key)) + "]",
equals(family, CellUtil.getFamilyArray(key)));
assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
"Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
equals(qualifier, key.getQualifier()));
"Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(key)) + "]",
equals(qualifier, CellUtil.getQualifierArray(key)));
assertTrue("Expected value [" + Bytes.toString(value) + "] " +
"Got value [" + Bytes.toString(key.getValue()) + "]",
equals(value, key.getValue()));
"Got value [" + Bytes.toString(CellUtil.getValueArray(key)) + "]",
equals(value, CellUtil.getValueArray(key)));
}
private void assertIncrementKey(KeyValue key, byte [] row, byte [] family,
private void assertIncrementKey(Cell key, byte [] row, byte [] family,
byte [] qualifier, long value)
throws Exception {
assertTrue("Expected row [" + Bytes.toString(row) + "] " +
"Got row [" + Bytes.toString(key.getRow()) +"]",
equals(row, key.getRow()));
"Got row [" + Bytes.toString(CellUtil.getRowArray(key)) +"]",
equals(row, CellUtil.getRowArray(key)));
assertTrue("Expected family [" + Bytes.toString(family) + "] " +
"Got family [" + Bytes.toString(key.getFamily()) + "]",
equals(family, key.getFamily()));
"Got family [" + Bytes.toString(CellUtil.getFamilyArray(key)) + "]",
equals(family, CellUtil.getFamilyArray(key)));
assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
"Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
equals(qualifier, key.getQualifier()));
"Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(key)) + "]",
equals(qualifier, CellUtil.getQualifierArray(key)));
assertTrue("Expected value [" + value + "] " +
"Got value [" + Bytes.toLong(key.getValue()) + "]",
Bytes.toLong(key.getValue()) == value);
"Got value [" + Bytes.toLong(CellUtil.getValueArray(key)) + "]",
Bytes.toLong(CellUtil.getValueArray(key)) == value);
}
private void assertNumKeys(Result result, int n) throws Exception {
@ -3131,23 +3133,26 @@ public class TestFromClientSide {
assertTrue("Expected " + idxs.length + " keys but result contains "
+ result.size(), result.size() == idxs.length);
KeyValue [] keys = result.raw();
Cell [] keys = result.raw();
for(int i=0;i<keys.length;i++) {
byte [] family = families[idxs[i][0]];
byte [] qualifier = qualifiers[idxs[i][1]];
byte [] value = values[idxs[i][2]];
KeyValue key = keys[i];
Cell key = keys[i];
byte[] famb = CellUtil.getFamilyArray(key);
byte[] qualb = CellUtil.getQualifierArray(key);
byte[] valb = CellUtil.getValueArray(key);
assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
+ "] " + "Got family [" + Bytes.toString(key.getFamily()) + "]",
equals(family, key.getFamily()));
+ "] " + "Got family [" + Bytes.toString(famb) + "]",
equals(family, famb));
assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
+ "] " + "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
equals(qualifier, key.getQualifier()));
+ "] " + "Got qualifier [" + Bytes.toString(qualb) + "]",
equals(qualifier, qualb));
assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
+ "Got value [" + Bytes.toString(key.getValue()) + "]",
equals(value, key.getValue()));
+ "Got value [" + Bytes.toString(valb) + "]",
equals(value, valb));
}
}
@ -3161,24 +3166,24 @@ public class TestFromClientSide {
int expectedResults = end - start + 1;
assertEquals(expectedResults, result.size());
KeyValue [] keys = result.raw();
Cell[] keys = result.raw();
for (int i=0; i<keys.length; i++) {
byte [] value = values[end-i];
long ts = stamps[end-i];
KeyValue key = keys[i];
Cell key = keys[i];
assertTrue("(" + i + ") Expected family [" + Bytes.toString(family)
+ "] " + "Got family [" + Bytes.toString(key.getFamily()) + "]",
equals(family, key.getFamily()));
+ "] " + "Got family [" + Bytes.toString(CellUtil.getFamilyArray(key)) + "]",
CellUtil.matchingFamily(key, family));
assertTrue("(" + i + ") Expected qualifier [" + Bytes.toString(qualifier)
+ "] " + "Got qualifier [" + Bytes.toString(key.getQualifier()) + "]",
equals(qualifier, key.getQualifier()));
+ "] " + "Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(key))+ "]",
CellUtil.matchingQualifier(key, qualifier));
assertTrue("Expected ts [" + ts + "] " +
"Got ts [" + key.getTimestamp() + "]", ts == key.getTimestamp());
assertTrue("(" + i + ") Expected value [" + Bytes.toString(value) + "] "
+ "Got value [" + Bytes.toString(key.getValue()) + "]",
equals(value, key.getValue()));
+ "Got value [" + Bytes.toString(CellUtil.getValueArray(key)) + "]",
CellUtil.matchingValue(key, value));
}
}
@ -3195,27 +3200,27 @@ public class TestFromClientSide {
equals(row, result.getRow()));
assertTrue("Expected two keys but result contains " + result.size(),
result.size() == 2);
KeyValue [] kv = result.raw();
KeyValue kvA = kv[0];
Cell [] kv = result.raw();
Cell kvA = kv[0];
assertTrue("(A) Expected family [" + Bytes.toString(familyA) + "] " +
"Got family [" + Bytes.toString(kvA.getFamily()) + "]",
equals(familyA, kvA.getFamily()));
"Got family [" + Bytes.toString(CellUtil.getFamilyArray(kvA)) + "]",
equals(familyA, CellUtil.getFamilyArray(kvA)));
assertTrue("(A) Expected qualifier [" + Bytes.toString(qualifierA) + "] " +
"Got qualifier [" + Bytes.toString(kvA.getQualifier()) + "]",
equals(qualifierA, kvA.getQualifier()));
"Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(kvA)) + "]",
equals(qualifierA, CellUtil.getQualifierArray(kvA)));
assertTrue("(A) Expected value [" + Bytes.toString(valueA) + "] " +
"Got value [" + Bytes.toString(kvA.getValue()) + "]",
equals(valueA, kvA.getValue()));
KeyValue kvB = kv[1];
"Got value [" + Bytes.toString(CellUtil.getValueArray(kvA)) + "]",
equals(valueA, CellUtil.getValueArray(kvA)));
Cell kvB = kv[1];
assertTrue("(B) Expected family [" + Bytes.toString(familyB) + "] " +
"Got family [" + Bytes.toString(kvB.getFamily()) + "]",
equals(familyB, kvB.getFamily()));
"Got family [" + Bytes.toString(CellUtil.getFamilyArray(kvB)) + "]",
equals(familyB, CellUtil.getFamilyArray(kvB)));
assertTrue("(B) Expected qualifier [" + Bytes.toString(qualifierB) + "] " +
"Got qualifier [" + Bytes.toString(kvB.getQualifier()) + "]",
equals(qualifierB, kvB.getQualifier()));
"Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(kvB)) + "]",
equals(qualifierB, CellUtil.getQualifierArray(kvB)));
assertTrue("(B) Expected value [" + Bytes.toString(valueB) + "] " +
"Got value [" + Bytes.toString(kvB.getValue()) + "]",
equals(valueB, kvB.getValue()));
"Got value [" + Bytes.toString(CellUtil.getValueArray(kvB)) + "]",
equals(valueB, CellUtil.getValueArray(kvB)));
}
private void assertSingleResult(Result result, byte [] row, byte [] family,
@ -3226,16 +3231,16 @@ public class TestFromClientSide {
equals(row, result.getRow()));
assertTrue("Expected a single key but result contains " + result.size(),
result.size() == 1);
KeyValue kv = result.raw()[0];
Cell kv = result.raw()[0];
assertTrue("Expected family [" + Bytes.toString(family) + "] " +
"Got family [" + Bytes.toString(kv.getFamily()) + "]",
equals(family, kv.getFamily()));
"Got family [" + Bytes.toString(CellUtil.getFamilyArray(kv)) + "]",
equals(family, CellUtil.getFamilyArray(kv)));
assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
"Got qualifier [" + Bytes.toString(kv.getQualifier()) + "]",
equals(qualifier, kv.getQualifier()));
"Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(kv)) + "]",
equals(qualifier, CellUtil.getQualifierArray(kv)));
assertTrue("Expected value [" + Bytes.toString(value) + "] " +
"Got value [" + Bytes.toString(kv.getValue()) + "]",
equals(value, kv.getValue()));
"Got value [" + Bytes.toString(CellUtil.getValueArray(kv)) + "]",
equals(value, CellUtil.getValueArray(kv)));
}
private void assertSingleResult(Result result, byte [] row, byte [] family,
@ -3246,18 +3251,18 @@ public class TestFromClientSide {
equals(row, result.getRow()));
assertTrue("Expected a single key but result contains " + result.size(),
result.size() == 1);
KeyValue kv = result.raw()[0];
Cell kv = result.raw()[0];
assertTrue("Expected family [" + Bytes.toString(family) + "] " +
"Got family [" + Bytes.toString(kv.getFamily()) + "]",
equals(family, kv.getFamily()));
"Got family [" + Bytes.toString(CellUtil.getFamilyArray(kv)) + "]",
equals(family, CellUtil.getFamilyArray(kv)));
assertTrue("Expected qualifier [" + Bytes.toString(qualifier) + "] " +
"Got qualifier [" + Bytes.toString(kv.getQualifier()) + "]",
equals(qualifier, kv.getQualifier()));
"Got qualifier [" + Bytes.toString(CellUtil.getQualifierArray(kv)) + "]",
equals(qualifier, CellUtil.getQualifierArray(kv)));
assertTrue("Expected ts [" + ts + "] " +
"Got ts [" + kv.getTimestamp() + "]", ts == kv.getTimestamp());
assertTrue("Expected value [" + Bytes.toString(value) + "] " +
"Got value [" + Bytes.toString(kv.getValue()) + "]",
equals(value, kv.getValue()));
"Got value [" + Bytes.toString(CellUtil.getValueArray(kv)) + "]",
equals(value, CellUtil.getValueArray(kv)));
}
private void assertEmptyResult(Result result) throws Exception {
@ -3809,7 +3814,7 @@ public class TestFromClientSide {
scan.addColumn(CONTENTS_FAMILY, null);
ResultScanner scanner = table.getScanner(scan);
for (Result r : scanner) {
for(KeyValue key : r.raw()) {
for(Cell key : r.raw()) {
System.out.println(Bytes.toString(r.getRow()) + ": " + key.toString());
}
}
@ -4011,7 +4016,7 @@ public class TestFromClientSide {
int index = 0;
Result r = null;
while ((r = s.next()) != null) {
for(KeyValue key : r.raw()) {
for(Cell key : r.raw()) {
times[index++] = key.getTimestamp();
}
}
@ -4045,7 +4050,7 @@ public class TestFromClientSide {
int index = 0;
Result r = null;
while ((r = s.next()) != null) {
for(KeyValue key : r.raw()) {
for(Cell key : r.raw()) {
times[index++] = key.getTimestamp();
}
}
@ -4172,7 +4177,7 @@ public class TestFromClientSide {
for (Result r : s) {
put = new Put(r.getRow());
put.setDurability(Durability.SKIP_WAL);
for (KeyValue kv : r.raw()) {
for (Cell kv : r.raw()) {
put.add(kv);
}
b.put(put);
@ -4521,7 +4526,7 @@ public class TestFromClientSide {
// Verify expected results
Result r = ht.get(new Get(ROW));
KeyValue [] kvs = r.raw();
Cell [] kvs = r.raw();
assertEquals(5, kvs.length);
assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 1);
assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 3);

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
@ -66,7 +67,7 @@ public class TestIntraRowPagination {
Result result;
boolean toLog = true;
List<KeyValue> kvListExp = new ArrayList<KeyValue>();
List<Cell> kvListExp = new ArrayList<Cell>();
int storeOffset = 1;
int storeLimit = 3;
@ -89,8 +90,8 @@ public class TestIntraRowPagination {
scan.setRowOffsetPerColumnFamily(storeOffset);
scan.setMaxResultsPerColumnFamily(storeLimit);
RegionScanner scanner = region.getScanner(scan);
List<KeyValue> kvListScan = new ArrayList<KeyValue>();
List<KeyValue> results = new ArrayList<KeyValue>();
List<Cell> kvListScan = new ArrayList<Cell>();
List<Cell> results = new ArrayList<Cell>();
while (scanner.next(results) || !results.isEmpty()) {
kvListScan.addAll(results);
results.clear();

View File

@ -32,8 +32,9 @@ import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ipc.RpcClient;
@ -183,12 +184,12 @@ public class TestMultiParallel {
Assert.assertEquals(singleRes.size(), multiRes.length);
for (int i = 0; i < singleRes.size(); i++) {
Assert.assertTrue(singleRes.get(i).containsColumn(BYTES_FAMILY, QUALIFIER));
KeyValue[] singleKvs = singleRes.get(i).raw();
KeyValue[] multiKvs = multiRes[i].raw();
Cell[] singleKvs = singleRes.get(i).raw();
Cell[] multiKvs = multiRes[i].raw();
for (int j = 0; j < singleKvs.length; j++) {
Assert.assertEquals(singleKvs[j], multiKvs[j]);
Assert.assertEquals(0, Bytes.compareTo(singleKvs[j].getValue(), multiKvs[j]
.getValue()));
Assert.assertEquals(0, Bytes.compareTo(CellUtil.getValueArray(singleKvs[j]),
CellUtil.getValueArray(multiKvs[j])));
}
}
table.close();

View File

@ -105,7 +105,7 @@ public class TestMultipleTimestamps {
ResultScanner scanner = scan(ht, FAMILY, scanRows, scanColumns,
scanTimestamps, scanMaxVersions);
KeyValue[] kvs;
Cell [] kvs;
kvs = scanner.next().raw();
assertEquals(2, kvs.length);
@ -147,7 +147,7 @@ public class TestMultipleTimestamps {
ResultScanner scanner = scan(ht, FAMILY, scanRows, scanColumns,
scanTimestamps, scanMaxVersions);
KeyValue[] kvs;
Cell[] kvs;
kvs = scanner.next().raw();
assertEquals(1, kvs.length);
@ -194,7 +194,7 @@ public class TestMultipleTimestamps {
}
scanner = scan(ht, FAMILY, scanRows, scanColumns, scanTimestamps, scanMaxVersions);
KeyValue[] kvs;
Cell[] kvs;
// This looks like wrong answer. Should be 2. Even then we are returning wrong result,
// timestamps that are 3 whereas should be 2 since min is inclusive.
@ -252,7 +252,7 @@ public class TestMultipleTimestamps {
ResultScanner scanner = scan(ht, FAMILY, scanRows, scanColumns,
scanTimestamps, scanMaxVersions);
KeyValue[] kvs;
Cell[] kvs;
kvs = scanner.next().raw();
assertEquals(2, kvs.length);
@ -307,7 +307,7 @@ public class TestMultipleTimestamps {
// request a bunch of versions including the deleted version. We should
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0,
Cell kvs[] = getNVersions(ht, FAMILY, 0, 0,
Arrays.asList(2L, 3L, 4L, 5L));
assertEquals(3, kvs.length);
checkOneCell(kvs[0], FAMILY, 0, 0, 5);
@ -339,7 +339,7 @@ public class TestMultipleTimestamps {
// request a bunch of versions including the deleted version. We should
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
Cell kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
assertEquals(0, kvs.length);
ht.close();
@ -365,7 +365,7 @@ public class TestMultipleTimestamps {
// request a bunch of versions including the deleted version. We should
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
Cell kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
assertEquals(0, kvs.length);
ht.close();
@ -391,7 +391,7 @@ public class TestMultipleTimestamps {
// request a bunch of versions including the deleted version. We should
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
Cell kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
assertEquals(0, kvs.length);
ht.close();
@ -401,26 +401,26 @@ public class TestMultipleTimestamps {
* Assert that the passed in KeyValue has expected contents for the
* specified row, column & timestamp.
*/
private void checkOneCell(KeyValue kv, byte[] cf,
private void checkOneCell(Cell kv, byte[] cf,
int rowIdx, int colIdx, long ts) {
String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
assertEquals("Row mismatch which checking: " + ctx,
"row:"+ rowIdx, Bytes.toString(kv.getRow()));
"row:"+ rowIdx, Bytes.toString(CellUtil.getRowArray(kv)));
assertEquals("ColumnFamily mismatch while checking: " + ctx,
Bytes.toString(cf), Bytes.toString(kv.getFamily()));
Bytes.toString(cf), Bytes.toString(CellUtil.getFamilyArray(kv)));
assertEquals("Column qualifier mismatch while checking: " + ctx,
"column:" + colIdx,
Bytes.toString(kv.getQualifier()));
Bytes.toString(CellUtil.getQualifierArray(kv)));
assertEquals("Timestamp mismatch while checking: " + ctx,
ts, kv.getTimestamp());
assertEquals("Value mismatch while checking: " + ctx,
"value-version-" + ts, Bytes.toString(kv.getValue()));
"value-version-" + ts, Bytes.toString(CellUtil.getValueArray(kv)));
}
/**
@ -428,7 +428,7 @@ public class TestMultipleTimestamps {
* versions for the row/column specified by rowIdx & colIdx.
*
*/
private KeyValue[] getNVersions(HTable ht, byte[] cf, int rowIdx,
private Cell[] getNVersions(HTable ht, byte[] cf, int rowIdx,
int colIdx, List<Long> versions)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);

View File

@ -19,21 +19,22 @@
package org.apache.hadoop.hbase.client;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.experimental.categories.Category;
import static org.apache.hadoop.hbase.HBaseTestCase.assertByteEquals;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestResult extends TestCase {
@ -69,9 +70,9 @@ public class TestResult extends TestCase {
for (int i = 0; i < 100; ++i) {
final byte[] qf = Bytes.toBytes(i);
List<KeyValue> ks = r.getColumn(family, qf);
List<Cell> ks = r.getColumn(family, qf);
assertEquals(1, ks.size());
assertByteEquals(qf, ks.get(0).getQualifier());
assertTrue(CellUtil.matchingQualifier(ks.get(0), qf));
assertEquals(ks.get(0), r.getColumnLatest(family, qf));
}
}
@ -90,9 +91,9 @@ public class TestResult extends TestCase {
for (int i = 0; i < 100; ++i) {
final byte[] qf = Bytes.toBytes(i);
List<KeyValue> ks = r.getColumn(family, qf);
List<Cell> ks = r.getColumn(family, qf);
assertEquals(2, ks.size());
assertByteEquals(qf, ks.get(0).getQualifier());
assertTrue(CellUtil.matchingQualifier(ks.get(0), qf));
assertEquals(200, ks.get(0).getTimestamp());
assertEquals(ks.get(0), r.getColumnLatest(family, qf));
}

View File

@ -24,6 +24,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTestConst;
import org.apache.hadoop.hbase.KeyValue;
@ -101,7 +102,7 @@ public class TestScannersFromClientSide {
Result result;
ResultScanner scanner;
boolean toLog = true;
List<KeyValue> kvListExp;
List<Cell> kvListExp;
// table: row, family, c0:0, c1:1, ... , c7:7
put = new Put(ROW);
@ -128,7 +129,7 @@ public class TestScannersFromClientSide {
scanner = ht.getScanner(scan);
// c4:4, c5:5, c6:6, c7:7
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[4], 4, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[5], 5, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[6], 6, VALUE));
@ -143,14 +144,14 @@ public class TestScannersFromClientSide {
scanner = ht.getScanner(scan);
// First batch: c4:4, c5:5
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[4], 4, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[5], 5, VALUE));
result = scanner.next();
verifyResult(result, kvListExp, toLog, "Testing first batch of scan");
// Second batch: c6:6, c7:7
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[6], 6, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[7], 7, VALUE));
result = scanner.next();
@ -175,9 +176,9 @@ public class TestScannersFromClientSide {
Put put;
Result result;
boolean toLog = true;
List<KeyValue> kvListExp;
List<Cell> kvListExp;
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
// Insert one CF for row[0]
put = new Put(ROW);
for (int i=0; i < 10; i++) {
@ -194,7 +195,7 @@ public class TestScannersFromClientSide {
get = new Get(ROW);
get.setMaxResultsPerColumnFamily(2);
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[0], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[1], 1, VALUE));
verifyResult(result, kvListExp, toLog, "Testing basic setMaxResults");
@ -205,7 +206,7 @@ public class TestScannersFromClientSide {
get.setFilter(new ColumnRangeFilter(QUALIFIERS[2], true, QUALIFIERS[5],
true));
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[2], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[3], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[4], 1, VALUE));
@ -233,7 +234,7 @@ public class TestScannersFromClientSide {
get.addFamily(FAMILIES[1]);
get.addFamily(FAMILIES[2]);
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
//Exp: CF1:q0, ..., q9, CF2: q0, q1, q10, q11, ..., q19
for (int i=0; i < 10; i++) {
kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE));
@ -251,7 +252,7 @@ public class TestScannersFromClientSide {
get.setMaxResultsPerColumnFamily(3);
get.setFilter(new ColumnRangeFilter(QUALIFIERS[2], true, null, true));
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
for (int i=2; i < 5; i++) {
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE));
}
@ -267,7 +268,7 @@ public class TestScannersFromClientSide {
get.setMaxResultsPerColumnFamily(7);
get.setFilter(new ColumnPrefixFilter(QUALIFIERS[1]));
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[1], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[1], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[1], 1, VALUE));
@ -296,9 +297,9 @@ public class TestScannersFromClientSide {
Scan scan;
Result result;
boolean toLog = true;
List<KeyValue> kvListExp, kvListScan;
List<Cell> kvListExp, kvListScan;
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
for (int r=0; r < ROWS.length; r++) {
put = new Put(ROWS[r]);
@ -317,9 +318,9 @@ public class TestScannersFromClientSide {
scan = new Scan();
scan.setMaxResultsPerColumnFamily(4);
ResultScanner scanner = ht.getScanner(scan);
kvListScan = new ArrayList<KeyValue>();
kvListScan = new ArrayList<Cell>();
while ((result = scanner.next()) != null) {
for (KeyValue kv : result.list()) {
for (Cell kv : result.list()) {
kvListScan.add(kv);
}
}
@ -345,10 +346,10 @@ public class TestScannersFromClientSide {
Put put;
Result result;
boolean toLog = true;
List<KeyValue> kvListExp;
List<Cell> kvListExp;
// Insert one CF for row
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
put = new Put(ROW);
for (int i=0; i < 10; i++) {
KeyValue kv = new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE);
@ -369,7 +370,7 @@ public class TestScannersFromClientSide {
get = new Get(ROW);
get.setRowOffsetPerColumnFamily(20);
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
verifyResult(result, kvListExp, toLog, "Testing offset > #kvs");
//offset + maxResultPerCF
@ -377,7 +378,7 @@ public class TestScannersFromClientSide {
get.setRowOffsetPerColumnFamily(4);
get.setMaxResultsPerColumnFamily(5);
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
for (int i=4; i < 9; i++) {
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE));
}
@ -390,7 +391,7 @@ public class TestScannersFromClientSide {
get.setFilter(new ColumnRangeFilter(QUALIFIERS[2], true, QUALIFIERS[5],
true));
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[3], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[4], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[5], 1, VALUE));
@ -413,7 +414,7 @@ public class TestScannersFromClientSide {
get.addFamily(FAMILIES[1]);
get.addFamily(FAMILIES[2]);
result = ht.get(get);
kvListExp = new ArrayList<KeyValue>();
kvListExp = new ArrayList<Cell>();
//Exp: CF1:q4, q5, CF2: q4, q5
kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[4], 1, VALUE));
kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[5], 1, VALUE));
@ -423,7 +424,7 @@ public class TestScannersFromClientSide {
"Testing offset + multiple CFs + maxResults");
}
static void verifyResult(Result result, List<KeyValue> expKvList, boolean toLog,
static void verifyResult(Result result, List<Cell> expKvList, boolean toLog,
String msg) {
LOG.info(msg);
@ -433,12 +434,12 @@ public class TestScannersFromClientSide {
return;
int i = 0;
for (KeyValue kv : result.raw()) {
for (Cell kv : result.raw()) {
if (i >= expKvList.size()) {
break; // we will check the size later
}
KeyValue kvExp = expKvList.get(i++);
Cell kvExp = expKvList.get(i++);
if (toLog) {
LOG.info("get kv is: " + kv.toString());
LOG.info("exp kv is: " + kvExp.toString());

View File

@ -93,7 +93,7 @@ public class TestTimestampsFilter {
byte [] TABLE = Bytes.toBytes("testTimestampsFilter");
byte [] FAMILY = Bytes.toBytes("event_log");
byte [][] FAMILIES = new byte[][] { FAMILY };
KeyValue kvs[];
Cell kvs[];
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
@ -169,7 +169,6 @@ public class TestTimestampsFilter {
byte [] TABLE = Bytes.toBytes("testTimestampsFilterMultiColumns");
byte [] FAMILY = Bytes.toBytes("event_log");
byte [][] FAMILIES = new byte[][] { FAMILY };
KeyValue kvs[];
// create table; set versions to max...
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES, Integer.MAX_VALUE);
@ -186,7 +185,7 @@ public class TestTimestampsFilter {
p.add(FAMILY, Bytes.toBytes("column4"), 3, Bytes.toBytes("value4-3"));
ht.put(p);
ArrayList timestamps = new ArrayList();
ArrayList<Long> timestamps = new ArrayList<Long>();
timestamps.add(new Long(3));
TimestampsFilter filter = new TimestampsFilter(timestamps);
@ -197,17 +196,15 @@ public class TestTimestampsFilter {
g.addColumn(FAMILY, Bytes.toBytes("column4"));
Result result = ht.get(g);
for (KeyValue kv : result.list()) {
System.out.println("found row " + Bytes.toString(kv.getRow()) +
", column " + Bytes.toString(kv.getQualifier()) + ", value "
+ Bytes.toString(kv.getValue()));
for (Cell kv : result.list()) {
System.out.println("found row " + Bytes.toString(CellUtil.getRowArray(kv)) +
", column " + Bytes.toString(CellUtil.getQualifierArray(kv)) + ", value "
+ Bytes.toString(CellUtil.getValueArray(kv)));
}
assertEquals(result.list().size(), 2);
assertEquals(Bytes.toString(result.list().get(0).getValue()),
"value2-3");
assertEquals(Bytes.toString(result.list().get(1).getValue()),
"value4-3");
assertTrue(CellUtil.matchingValue(result.list().get(0), Bytes.toBytes("value2-3")));
assertTrue(CellUtil.matchingValue(result.list().get(1), Bytes.toBytes("value4-3")));
ht.close();
}
@ -248,7 +245,7 @@ public class TestTimestampsFilter {
// request a bunch of versions including the deleted version. We should
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L, 4L, 5L));
Cell kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L, 4L, 5L));
assertEquals(3, kvs.length);
checkOneCell(kvs[0], FAMILY, 0, 0, 5);
checkOneCell(kvs[1], FAMILY, 0, 0, 3);
@ -261,7 +258,7 @@ public class TestTimestampsFilter {
for (int rowIdx = 0; rowIdx < 5; rowIdx++) {
for (int colIdx = 0; colIdx < 5; colIdx++) {
// ask for versions that exist.
KeyValue[] kvs = getNVersions(ht, cf, rowIdx, colIdx,
Cell[] kvs = getNVersions(ht, cf, rowIdx, colIdx,
Arrays.asList(5L, 300L, 6L, 80L));
assertEquals(4, kvs.length);
checkOneCell(kvs[0], cf, rowIdx, colIdx, 300);
@ -289,26 +286,26 @@ public class TestTimestampsFilter {
* Assert that the passed in KeyValue has expected contents for the
* specified row, column & timestamp.
*/
private void checkOneCell(KeyValue kv, byte[] cf,
private void checkOneCell(Cell kv, byte[] cf,
int rowIdx, int colIdx, long ts) {
String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
assertEquals("Row mismatch which checking: " + ctx,
"row:"+ rowIdx, Bytes.toString(kv.getRow()));
"row:"+ rowIdx, Bytes.toString(CellUtil.getRowArray(kv)));
assertEquals("ColumnFamily mismatch while checking: " + ctx,
Bytes.toString(cf), Bytes.toString(kv.getFamily()));
Bytes.toString(cf), Bytes.toString(CellUtil.getFamilyArray(kv)));
assertEquals("Column qualifier mismatch while checking: " + ctx,
"column:" + colIdx,
Bytes.toString(kv.getQualifier()));
Bytes.toString(CellUtil.getQualifierArray(kv)));
assertEquals("Timestamp mismatch while checking: " + ctx,
ts, kv.getTimestamp());
assertEquals("Value mismatch while checking: " + ctx,
"value-version-" + ts, Bytes.toString(kv.getValue()));
"value-version-" + ts, Bytes.toString(CellUtil.getValueArray(kv)));
}
/**
@ -316,7 +313,7 @@ public class TestTimestampsFilter {
* versions for the row/column specified by rowIdx & colIdx.
*
*/
private KeyValue[] getNVersions(HTable ht, byte[] cf, int rowIdx,
private Cell[] getNVersions(HTable ht, byte[] cf, int rowIdx,
int colIdx, List<Long> versions)
throws IOException {
byte row[] = Bytes.toBytes("row:" + rowIdx);

View File

@ -24,6 +24,8 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.KeyValue;
@ -83,14 +85,14 @@ implements Coprocessor, CoprocessorService {
InternalScanner scanner = null;
try {
scanner = this.env.getRegion().getScanner(scan);
List<KeyValue> curVals = new ArrayList<KeyValue>();
List<Cell> curVals = new ArrayList<Cell>();
boolean hasMore = false;
do {
curVals.clear();
hasMore = scanner.next(curVals);
for (KeyValue kv : curVals) {
if (Bytes.equals(qualifier, kv.getQualifier())) {
sumResult += Bytes.toInt(kv.getBuffer(), kv.getValueOffset());
for (Cell kv : curVals) {
if (CellUtil.matchingQualifier(kv, qualifier)) {
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());
}
}
} while (hasMore);

View File

@ -34,6 +34,7 @@ import com.google.common.collect.ImmutableList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get;
@ -270,8 +271,8 @@ public class SimpleRegionObserver extends BaseRegionObserver {
}
@Override
public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> results) throws IOException {
public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<Cell> results) throws IOException {
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
assertNotNull(e.getRegion());
@ -281,8 +282,8 @@ public class SimpleRegionObserver extends BaseRegionObserver {
}
@Override
public void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> results) {
public void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<Cell> results) {
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
assertNotNull(e.getRegion());
@ -293,14 +294,14 @@ public class SimpleRegionObserver extends BaseRegionObserver {
boolean foundA = false;
boolean foundB = false;
boolean foundC = false;
for (KeyValue kv: results) {
if (Bytes.equals(kv.getFamily(), TestRegionObserverInterface.A)) {
for (Cell kv: results) {
if (CellUtil.matchingFamily(kv, TestRegionObserverInterface.A)) {
foundA = true;
}
if (Bytes.equals(kv.getFamily(), TestRegionObserverInterface.B)) {
if (CellUtil.matchingFamily(kv, TestRegionObserverInterface.B)) {
foundB = true;
}
if (Bytes.equals(kv.getFamily(), TestRegionObserverInterface.C)) {
if (CellUtil.matchingFamily(kv, TestRegionObserverInterface.C)) {
foundC = true;
}
}

View File

@ -39,6 +39,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestCase;
@ -47,7 +48,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
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.SmallTests;
import org.apache.hadoop.hbase.TableName;
@ -84,23 +84,23 @@ public class TestCoprocessorInterface {
}
@Override
public boolean next(List<KeyValue> results) throws IOException {
public boolean next(List<Cell> results) throws IOException {
return delegate.next(results);
}
@Override
public boolean next(List<KeyValue> result, int limit) throws IOException {
public boolean next(List<Cell> result, int limit) throws IOException {
return delegate.next(result, limit);
}
@Override
public boolean nextRaw(List<KeyValue> result)
public boolean nextRaw(List<Cell> result)
throws IOException {
return delegate.nextRaw(result);
}
@Override
public boolean nextRaw(List<KeyValue> result, int limit)
public boolean nextRaw(List<Cell> result, int limit)
throws IOException {
return delegate.nextRaw(result, limit);
}
@ -261,8 +261,8 @@ public class TestCoprocessorInterface {
sharedData = null;
}
@Override
public void preGet(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final List<KeyValue> results) throws IOException {
public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final List<Cell> results) throws IOException {
if (1/0 == 1) {
e.complete();
}
@ -382,7 +382,7 @@ public class TestCoprocessorInterface {
RegionScanner scanner = regions[0].getCoprocessorHost().postScannerOpen(s, regions[0].getScanner(s));
assertTrue(scanner instanceof CustomScanner);
// this would throw an exception before HBASE-4197
scanner.next(new ArrayList<KeyValue>());
scanner.next(new ArrayList<Cell>());
assertTrue("Coprocessor not started", ((CoprocessorImpl)c).wasStarted());
assertTrue("Coprocessor not stopped", ((CoprocessorImpl)c).wasStopped());

View File

@ -34,6 +34,8 @@ 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.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -333,19 +335,19 @@ public class TestRegionObserverInterface {
Store store, final InternalScanner scanner, final ScanType scanType) {
return new InternalScanner() {
@Override
public boolean next(List<KeyValue> results) throws IOException {
public boolean next(List<Cell> results) throws IOException {
return next(results, -1);
}
@Override
public boolean next(List<KeyValue> results, int limit)
public boolean next(List<Cell> results, int limit)
throws IOException{
List<KeyValue> internalResults = new ArrayList<KeyValue>();
List<Cell> internalResults = new ArrayList<Cell>();
boolean hasMore;
do {
hasMore = scanner.next(internalResults, limit);
if (!internalResults.isEmpty()) {
long row = Bytes.toLong(internalResults.get(0).getRow());
long row = Bytes.toLong(CellUtil.getValueArray(internalResults.get(0)));
if (row % 2 == 0) {
// return this row
break;

Some files were not shown because too many files have changed in this diff Show More