HBASE-9164 Convert List<? extends Cell> anti pattern to List<Cell> pattern

This patch also starts a refactor of the KVComparator.


git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1514400 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-08-15 17:49:06 +00:00
parent 58ee2bbf24
commit 4dd82c9b1c
26 changed files with 257 additions and 161 deletions

View File

@ -105,15 +105,12 @@ public class Append extends Mutation {
// Presume it is KeyValue for now.
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
byte [] family = kv.getFamily();
List<? extends Cell> list = this.familyMap.get(family);
List<Cell> list = this.familyMap.get(family);
if (list == null) {
list = new ArrayList<Cell>();
}
// Cast so explicit list type rather than ? extends Cell. Help the compiler out. See
// http://stackoverflow.com/questions/6474784/java-using-generics-with-lists-and-interfaces
List<KeyValue> l = (List<KeyValue>)list;
// find where the new entry should be placed in the List
l.add(kv);
list.add(kv);
this.familyMap.put(family, list);
return this;
}

View File

@ -161,13 +161,11 @@ public class Delete extends Mutation implements Comparable<Row> {
" doesn't match the original one " + Bytes.toStringBinary(this.row));
}
byte [] family = kv.getFamily();
List<? extends Cell> list = familyMap.get(family);
List<Cell> list = familyMap.get(family);
if (list == null) {
list = new ArrayList<Cell>();
}
// Cast so explicit list type rather than ? extends Cell. Help the compiler out. See
// http://stackoverflow.com/questions/6474784/java-using-generics-with-lists-and-interfaces
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(family, list);
return this;
}
@ -197,16 +195,14 @@ public class Delete extends Mutation implements Comparable<Row> {
*/
@SuppressWarnings("unchecked")
public Delete deleteFamily(byte [] family, long timestamp) {
List<? extends Cell> list = familyMap.get(family);
List<Cell> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<Cell>();
} else if(!list.isEmpty()) {
list.clear();
}
KeyValue kv = new KeyValue(row, family, null, timestamp, KeyValue.Type.DeleteFamily);
// Cast so explicit list type rather than ? extends Cell. Help the compiler out. See
// http://stackoverflow.com/questions/6474784/java-using-generics-with-lists-and-interfaces
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(family, list);
return this;
}
@ -219,11 +215,11 @@ public class Delete extends Mutation implements Comparable<Row> {
* @return this for invocation chaining
*/
public Delete deleteFamilyVersion(byte [] family, long timestamp) {
List<? extends Cell> list = familyMap.get(family);
List<Cell> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<Cell>();
}
((List<KeyValue>)list).add(new KeyValue(row, family, null, timestamp,
list.add(new KeyValue(row, family, null, timestamp,
KeyValue.Type.DeleteFamilyVersion));
familyMap.put(family, list);
return this;
@ -251,13 +247,11 @@ public class Delete extends Mutation implements Comparable<Row> {
*/
@SuppressWarnings("unchecked")
public Delete deleteColumns(byte [] family, byte [] qualifier, long timestamp) {
List<? extends Cell> list = familyMap.get(family);
List<Cell> list = familyMap.get(family);
if (list == null) {
list = new ArrayList<Cell>();
}
// Cast so explicit list type rather than ? extends Cell. Help the compiler out. See
// http://stackoverflow.com/questions/6474784/java-using-generics-with-lists-and-interfaces
((List<KeyValue>)list).add(new KeyValue(this.row, family, qualifier, timestamp,
list.add(new KeyValue(this.row, family, qualifier, timestamp,
KeyValue.Type.DeleteColumn));
familyMap.put(family, list);
return this;
@ -286,14 +280,12 @@ public class Delete extends Mutation implements Comparable<Row> {
*/
@SuppressWarnings("unchecked")
public Delete deleteColumn(byte [] family, byte [] qualifier, long timestamp) {
List<? extends Cell> list = familyMap.get(family);
List<Cell> list = familyMap.get(family);
if(list == null) {
list = new ArrayList<Cell>();
}
// Cast so explicit list type rather than ? extends Cell. Help the compiler out. See
// http://stackoverflow.com/questions/6474784/java-using-generics-with-lists-and-interfaces
KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp, KeyValue.Type.Delete);
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(family, list);
return this;
}

View File

@ -1328,7 +1328,7 @@ public class HTable implements HTableInterface {
throw new IllegalArgumentException("No columns to insert");
}
if (maxKeyValueSize > 0) {
for (List<? extends Cell> list : put.getFamilyCellMap().values()) {
for (List<Cell> list : put.getFamilyCellMap().values()) {
for (Cell cell : list) {
// KeyValue v1 expectation. Cast for now.
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);

View File

@ -83,7 +83,7 @@ public class Increment extends Mutation implements Comparable<Row> {
public Increment add(Cell cell) throws IOException{
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
byte [] family = kv.getFamily();
List<? extends Cell> list = getCellList(family);
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());
@ -91,7 +91,7 @@ public class Increment extends Mutation implements Comparable<Row> {
throw new WrongRowIOException("The row in " + kv.toString() +
" doesn't match the original one " + Bytes.toStringBinary(this.row));
}
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(family, list);
return this;
}
@ -114,9 +114,9 @@ public class Increment extends Mutation implements Comparable<Row> {
if (qualifier == null) {
throw new IllegalArgumentException("qualifier cannot be null");
}
List<? extends Cell> list = getCellList(family);
List<Cell> list = getCellList(family);
KeyValue kv = createPutKeyValue(family, qualifier, ts, Bytes.toBytes(amount));
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(kv.getFamily(), list);
return this;
}
@ -174,10 +174,10 @@ public class Increment extends Mutation implements Comparable<Row> {
* @since 0.95.0
*/
public Map<byte[], NavigableMap<byte [], Long>> getFamilyMapOfLongs() {
NavigableMap<byte[], List<? extends Cell>> map = super.getFamilyCellMap();
NavigableMap<byte[], List<Cell>> map = super.getFamilyCellMap();
Map<byte [], NavigableMap<byte[], Long>> results =
new TreeMap<byte[], NavigableMap<byte [], Long>>(Bytes.BYTES_COMPARATOR);
for (Map.Entry<byte [], List<? extends Cell>> entry: map.entrySet()) {
for (Map.Entry<byte [], List<Cell>> entry: map.entrySet()) {
NavigableMap<byte [], Long> longs = new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
for (Cell cell: entry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
@ -202,7 +202,7 @@ public class Increment extends Mutation implements Comparable<Row> {
}
sb.append(", families=");
boolean moreThanOne = false;
for(Map.Entry<byte [], List<? extends Cell>> entry: this.familyMap.entrySet()) {
for(Map.Entry<byte [], List<Cell>> entry: this.familyMap.entrySet()) {
if(moreThanOne) {
sb.append("), ");
} else {

View File

@ -65,8 +65,8 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
protected Durability durability = Durability.USE_DEFAULT;
// A Map sorted by column family.
protected NavigableMap<byte [], List<? extends Cell>> familyMap =
new TreeMap<byte [], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
protected NavigableMap<byte [], List<Cell>> familyMap =
new TreeMap<byte [], List<Cell>>(Bytes.BYTES_COMPARATOR);
@Override
public CellScanner cellScanner() {
@ -80,8 +80,8 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
* @param family column family
* @return a list of Cell objects, returns an empty list if one doesn't exist.
*/
List<? extends Cell> getCellList(byte[] family) {
List<? extends Cell> list = this.familyMap.get(family);
List<Cell> getCellList(byte[] family) {
List<Cell> list = this.familyMap.get(family);
if (list == null) {
list = new ArrayList<Cell>();
}
@ -110,7 +110,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
// ideally, we would also include table information, but that information
// is not stored in each Operation instance.
map.put("families", families);
for (Map.Entry<byte [], List<? extends Cell>> entry : this.familyMap.entrySet()) {
for (Map.Entry<byte [], List<Cell>> entry : this.familyMap.entrySet()) {
families.add(Bytes.toStringBinary(entry.getKey()));
}
return map;
@ -135,7 +135,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
map.put("row", Bytes.toStringBinary(this.row));
int colCount = 0;
// iterate through all column families affected
for (Map.Entry<byte [], List<? extends Cell>> entry : this.familyMap.entrySet()) {
for (Map.Entry<byte [], List<Cell>> entry : this.familyMap.entrySet()) {
// map from this family to details for each cell affected within the family
List<Map<String, Object>> qualifierDetails = new ArrayList<Map<String, Object>>();
columns.put(Bytes.toStringBinary(entry.getKey()), qualifierDetails);
@ -182,14 +182,14 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
* Method for retrieving the put's familyMap
* @return familyMap
*/
public NavigableMap<byte [], List<? extends Cell>> getFamilyCellMap() {
public NavigableMap<byte [], List<Cell>> getFamilyCellMap() {
return this.familyMap;
}
/**
* Method for setting the put's familyMap
*/
public void setFamilyMap(NavigableMap<byte [], List<? extends Cell>> map) {
public void setFamilyMap(NavigableMap<byte [], List<Cell>> map) {
// TODO: Shut this down or move it up to be a Constructor. Get new object rather than change
// this internal data member.
this.familyMap = map;
@ -253,7 +253,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
*/
public int size() {
int size = 0;
for (List<? extends Cell> cells : this.familyMap.values()) {
for (List<Cell> cells : this.familyMap.values()) {
size += cells.size();
}
return size;
@ -278,7 +278,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
// Adding map overhead
heapsize +=
ClassSize.align(this.familyMap.size() * ClassSize.MAP_ENTRY);
for(Map.Entry<byte [], List<? extends Cell>> entry : this.familyMap.entrySet()) {
for(Map.Entry<byte [], List<Cell>> entry : this.familyMap.entrySet()) {
//Adding key overhead
heapsize +=
ClassSize.align(ClassSize.ARRAY + entry.getKey().length);

View File

@ -92,8 +92,8 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
*/
public Put(Put putToCopy) {
this(putToCopy.getRow(), putToCopy.ts);
this.familyMap = new TreeMap<byte [], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte [], List<? extends Cell>> entry: putToCopy.getFamilyCellMap().entrySet()) {
this.familyMap = new TreeMap<byte [], List<Cell>>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte [], List<Cell>> entry: putToCopy.getFamilyCellMap().entrySet()) {
this.familyMap.put(entry.getKey(), entry.getValue());
}
this.durability = putToCopy.durability;
@ -121,9 +121,9 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
*/
@SuppressWarnings("unchecked")
public Put add(byte [] family, byte [] qualifier, long ts, byte [] value) {
List<? extends Cell> list = getCellList(family);
List<Cell> list = getCellList(family);
KeyValue kv = createPutKeyValue(family, qualifier, ts, value);
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(kv.getFamily(), list);
return this;
}
@ -139,7 +139,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
@SuppressWarnings("unchecked")
public Put add(KeyValue kv) throws IOException{
byte [] family = kv.getFamily();
List<? extends Cell> list = getCellList(family);
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());
@ -147,7 +147,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
throw new WrongRowIOException("The row in " + kv.toString() +
" doesn't match the original one " + Bytes.toStringBinary(this.row));
}
((List<KeyValue>)list).add(kv);
list.add(kv);
familyMap.put(family, list);
return this;
}
@ -228,7 +228,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
*/
private boolean has(byte[] family, byte[] qualifier, long ts, byte[] value,
boolean ignoreTS, boolean ignoreValue) {
List<? extends Cell> list = getCellList(family);
List<Cell> list = getCellList(family);
if (list.size() == 0) {
return false;
}

View File

@ -927,10 +927,10 @@ public final class ProtobufUtil {
}
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[], List<? extends Cell>> family: increment.getFamilyCellMap().entrySet()) {
for (Map.Entry<byte[], List<Cell>> family: increment.getFamilyCellMap().entrySet()) {
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
columnBuilder.clearQualifierValue();
List<? extends Cell> values = family.getValue();
List<Cell> values = family.getValue();
if (values != null && values.size() > 0) {
for (Cell cell: values) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
@ -957,7 +957,7 @@ public final class ProtobufUtil {
MutationProto.Builder builder = getMutationBuilderAndSetCommonFields(type, mutation);
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<? extends Cell>> family: mutation.getFamilyCellMap().entrySet()) {
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
columnBuilder.clearQualifierValue();
for (Cell cell: family.getValue()) {

View File

@ -226,11 +226,11 @@ public final class CellUtil {
* @return CellScanner interface over <code>cellIterable</code>
*/
public static CellScanner createCellScanner(final NavigableMap<byte [],
List<? extends Cell>> map) {
List<Cell>> map) {
return new CellScanner() {
private final Iterator<Entry<byte[], List<? extends Cell>>> entries =
private final Iterator<Entry<byte[], List<Cell>>> entries =
map.entrySet().iterator();
private Iterator<? extends Cell> currentIterator = null;
private Iterator<Cell> currentIterator = null;
private Cell currentCell;
@Override

View File

@ -97,12 +97,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
*/
public static final KVComparator META_COMPARATOR = new MetaComparator();
/**
* A {@link KVComparator} for <code>.META.</code> catalog table
* {@link KeyValue} keys.
*/
public static final KeyComparator META_KEY_COMPARATOR = new MetaKeyComparator();
/**
* Get the appropriate row comparator for the specified table.
*
@ -257,16 +251,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
this.memstoreTS = mvccVersion;
}
@Deprecated
public long getMemstoreTS() {
return getMvccVersion();
}
@Deprecated
public void setMemstoreTS(long memstoreTS) {
setMvccVersion(memstoreTS);
}
// default value is 0, aka DNC
private long memstoreTS = 0;
@ -871,7 +855,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
// Important to clone the memstoreTS as well - otherwise memstore's
// update-in-place methods (eg increment) will end up creating
// new entries
ret.setMemstoreTS(memstoreTS);
ret.setMvccVersion(memstoreTS);
return ret;
}
@ -882,7 +866,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
*/
public KeyValue shallowCopy() {
KeyValue shallowCopy = new KeyValue(this.bytes, this.offset, this.length);
shallowCopy.setMemstoreTS(this.memstoreTS);
shallowCopy.setMvccVersion(this.memstoreTS);
return shallowCopy;
}
@ -1779,6 +1763,61 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
protected Object clone() throws CloneNotSupportedException {
return new MetaComparator();
}
/**
* Override the row key comparision to parse and compare the meta row key parts.
*/
@Override
protected int compareRowKey(final Cell l, final Cell r) {
byte[] left = l.getRowArray();
int loffset = l.getRowOffset();
int llength = l.getRowLength();
byte[] right = r.getRowArray();
int roffset = r.getRowOffset();
int rlength = r.getRowLength();
int leftDelimiter = getDelimiter(left, loffset, llength,
HConstants.DELIMITER);
int rightDelimiter = getDelimiter(right, roffset, rlength,
HConstants.DELIMITER);
if (leftDelimiter < 0 && rightDelimiter >= 0) {
// Nothing between .META. and regionid. Its first key.
return -1;
} else if (rightDelimiter < 0 && leftDelimiter >= 0) {
return 1;
} else if (leftDelimiter < 0 && rightDelimiter < 0) {
return 0;
}
// Compare up to the delimiter
int result = Bytes.compareTo(left, loffset, leftDelimiter - loffset,
right, roffset, rightDelimiter - roffset);
if (result != 0) {
return result;
}
// Compare middle bit of the row.
// Move past delimiter
leftDelimiter++;
rightDelimiter++;
int leftFarDelimiter = getRequiredDelimiterInReverse(left, leftDelimiter,
llength - (leftDelimiter - loffset), HConstants.DELIMITER);
int rightFarDelimiter = getRequiredDelimiterInReverse(right,
rightDelimiter, rlength - (rightDelimiter - roffset),
HConstants.DELIMITER);
// Now compare middlesection of row.
result = Bytes.compareTo(
left, leftDelimiter, leftFarDelimiter - leftDelimiter,
right, rightDelimiter, rightFarDelimiter - rightDelimiter);
if (result != 0) {
return result;
}
// Compare last part of row, the rowid.
leftFarDelimiter++;
rightFarDelimiter++;
result = Bytes.compareTo(
left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
return result;
}
}
/**
@ -1787,7 +1826,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* considered the same as far as this Comparator is concerned.
* Hosts a {@link KeyComparator}.
*/
public static class KVComparator implements java.util.Comparator<KeyValue> {
public static class KVComparator implements java.util.Comparator<Cell> {
private final KeyComparator rawcomparator = new KeyComparator();
/**
@ -1798,12 +1837,81 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return this.rawcomparator;
}
public int compare(final KeyValue left, final KeyValue right) {
int ret = getRawComparator().compare(left.getBuffer(),
left.getOffset() + ROW_OFFSET, left.getKeyLength(),
right.getBuffer(), right.getOffset() + ROW_OFFSET,
right.getKeyLength());
if (ret != 0) return ret;
protected int compareRowKey(final Cell left, final Cell right) {
return Bytes.compareTo(
left.getRowArray(), left.getRowOffset(), left.getRowLength(),
right.getRowArray(), right.getRowOffset(), right.getRowLength());
}
/**
* Compares the Key of a cell -- with fields being more significant in this order:
* rowkey, colfam/qual, timestamp, type, mvcc
*/
public int compare(final Cell left, final Cell right) {
// compare row
int compare = compareRowKey(left, right);
if (compare != 0) {
return compare;
}
// compare vs minimum
byte ltype = left.getTypeByte();
byte rtype = right.getTypeByte();
// If the column is not specified, the "minimum" key type appears the
// latest in the sorted order, regardless of the timestamp. This is used
// for specifying the last key/value in a given row, because there is no
// "lexicographically last column" (it would be infinitely long). The
// "maximum" key type does not need this behavior.
int lcfqLen = left.getFamilyLength() + left.getQualifierLength() ;
int rcfqLen = right.getFamilyLength() + right.getQualifierLength() ;
if (lcfqLen == 0 && ltype == Type.Minimum.getCode()) {
// left is "bigger", i.e. it appears later in the sorted order
return 1;
}
if (rcfqLen == 0 && rtype == Type.Minimum.getCode()) {
return -1;
}
// compare col family / col fam + qual
// If left family size is not equal to right family size, we need not
// compare the qualifiers.
compare = Bytes.compareTo(
left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
if (compare != 0) {
return compare;
}
// Compare qualifier
compare = Bytes.compareTo(
left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
if (compare!= 0) {
return compare;
}
// compare timestamp
long ltimestamp = left.getTimestamp();
long rtimestamp = right.getTimestamp();
compare = KeyComparator.compareTimestamps(ltimestamp, rtimestamp);
if (compare != 0) {
return compare;
}
// Compare types. Let the delete types sort ahead of puts; i.e. types
// of higher numbers sort before those of lesser numbers. Maximum (255)
// appears ahead of everything, and minimum (0) appears after
// everything.
compare = (0xff & rtype) - (0xff & ltype);
if (compare != 0) {
return compare;
}
// compare Mvcc Version
// Negate this comparison so later edits show up first
return -Longs.compare(left.getMvccVersion(), right.getMvccVersion());
}
@ -1818,7 +1926,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
// Compare timestamps
long ltimestamp = left.getTimestamp(lkeylength);
long rtimestamp = right.getTimestamp(rkeylength);
return getRawComparator().compareTimestamps(ltimestamp, rtimestamp);
return KeyComparator.compareTimestamps(ltimestamp, rtimestamp);
}
/**
@ -1964,7 +2072,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
protected Object clone() throws CloneNotSupportedException {
return new KVComparator();
}
}
/**
@ -2353,8 +2460,6 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
public static class MetaKeyComparator extends KeyComparator {
public int compareRows(byte [] left, int loffset, int llength,
byte [] right, int roffset, int rlength) {
// LOG.info("META " + Bytes.toString(left, loffset, llength) +
// "---" + Bytes.toString(right, roffset, rlength));
int leftDelimiter = getDelimiter(left, loffset, llength,
HConstants.DELIMITER);
int rightDelimiter = getDelimiter(right, roffset, rlength,
@ -2681,7 +2786,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
right, roffset, rlength, rfamilylength);
}
int compareTimestamps(final long ltimestamp, final long rtimestamp) {
static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
// The below older timestamps sorting ahead of newer timestamps looks
// wrong but it is intentional. This way, newer timestamps are first
// found when we iterate over a memstore and newer versions are the

View File

@ -56,17 +56,17 @@ public class PutCombiner<K> extends Reducer<K, Put, K, Put> {
int cnt = 0;
long curSize = 0;
Put put = null;
Map<byte[], List<? extends Cell>> familyMap = null;
Map<byte[], List<Cell>> familyMap = null;
for (Put p : vals) {
cnt++;
if (put == null) {
put = p;
familyMap = put.getFamilyCellMap();
} else {
for (Entry<byte[], List<? extends Cell>> entry : p.getFamilyCellMap()
for (Entry<byte[], List<Cell>> entry : p.getFamilyCellMap()
.entrySet()) {
List<? extends Cell> cells = familyMap.get(entry.getKey());
List<KeyValue> kvs = (cells != null) ? (List<KeyValue>) cells : null;
List<Cell> cells = familyMap.get(entry.getKey());
List<Cell> kvs = (cells != null) ? (List<Cell>) cells : null;
for (Cell cell : entry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
curSize += kv.heapSize();

View File

@ -63,7 +63,7 @@ public class PutSortReducer extends
// stop at the end or the RAM threshold
while (iter.hasNext() && curSize < threshold) {
Put p = iter.next();
for (List<? extends Cell> cells: p.getFamilyCellMap().values()) {
for (List<Cell> cells: p.getFamilyCellMap().values()) {
for (Cell cell: cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
map.add(kv);

View File

@ -29,6 +29,7 @@ import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.UUID;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.KeyValue;
@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
public class ReplicationProtbufUtil {
/**
* Get the HLog entries from a list of protocol buffer WALEntry

View File

@ -1782,7 +1782,7 @@ public class HRegion implements HeapSize { // , Writable{
* @param durability
* @throws IOException
*/
void delete(NavigableMap<byte[], List<? extends Cell>> familyMap, UUID clusterId,
void delete(NavigableMap<byte[], List<Cell>> familyMap, UUID clusterId,
Durability durability) throws IOException {
Delete delete = new Delete(FOR_UNIT_TESTS_ONLY);
delete.setFamilyMap(familyMap);
@ -1798,12 +1798,12 @@ public class HRegion implements HeapSize { // , Writable{
* @param byteNow
* @throws IOException
*/
void prepareDeleteTimestamps(Map<byte[], List<? extends Cell>> familyMap, byte[] byteNow)
void prepareDeleteTimestamps(Map<byte[], List<Cell>> familyMap, byte[] byteNow)
throws IOException {
for (Map.Entry<byte[], List<? extends Cell>> e : familyMap.entrySet()) {
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
byte[] family = e.getKey();
List<? extends Cell> cells = e.getValue();
List<Cell> cells = e.getValue();
Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
for (Cell cell: cells) {
@ -2009,7 +2009,7 @@ public class HRegion implements HeapSize { // , Writable{
/** Keep track of the locks we hold so we can release them in finally clause */
List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
// reference family maps directly so coprocessors can mutate them if desired
Map<byte[], List<? extends Cell>>[] familyMaps = new Map[batchOp.operations.length];
Map<byte[], List<Cell>>[] familyMaps = new Map[batchOp.operations.length];
// We try to set up a batch in the range [firstIndex,lastIndexExclusive)
int firstIndex = batchOp.nextIndexToProcess;
int lastIndexExclusive = firstIndex;
@ -2026,7 +2026,7 @@ public class HRegion implements HeapSize { // , Writable{
Mutation mutation = batchOp.operations[lastIndexExclusive];
boolean isPutMutation = mutation instanceof Put;
Map<byte[], List<? extends Cell>> familyMap = mutation.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
// store the family map reference to allow for mutations
familyMaps[lastIndexExclusive] = familyMap;
@ -2488,8 +2488,8 @@ public class HRegion implements HeapSize { // , Writable{
* Replaces any KV timestamps set to {@link HConstants#LATEST_TIMESTAMP} with the
* provided current timestamp.
*/
void updateKVTimestamps(final Iterable<List<? extends Cell>> keyLists, final byte[] now) {
for (List<? extends Cell> cells: keyLists) {
void updateKVTimestamps(final Iterable<List<Cell>> keyLists, final byte[] now) {
for (List<Cell> cells: keyLists) {
if (cells == null) continue;
for (Cell cell : cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
@ -2581,10 +2581,10 @@ public class HRegion implements HeapSize { // , Writable{
* @praram now
* @throws IOException
*/
private void put(final byte [] row, byte [] family, List<? extends Cell> edits)
private void put(final byte [] row, byte [] family, List<Cell> edits)
throws IOException {
NavigableMap<byte[], List<? extends Cell>> familyMap;
familyMap = new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
NavigableMap<byte[], List<Cell>> familyMap;
familyMap = new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
familyMap.put(family, edits);
Put p = new Put(row);
@ -2605,7 +2605,7 @@ public class HRegion implements HeapSize { // , Writable{
* @return the additional memory usage of the memstore caused by the
* new entries.
*/
private long applyFamilyMapToMemstore(Map<byte[], List<? extends Cell>> familyMap,
private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap,
MultiVersionConsistencyControl.WriteEntry localizedWriteEntry) {
long size = 0;
boolean freemvcc = false;
@ -2616,9 +2616,9 @@ public class HRegion implements HeapSize { // , Writable{
freemvcc = true;
}
for (Map.Entry<byte[], List<? extends Cell>> e : familyMap.entrySet()) {
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
byte[] family = e.getKey();
List<? extends Cell> cells = e.getValue();
List<Cell> cells = e.getValue();
Store store = getStore(family);
for (Cell cell: cells) {
@ -2642,7 +2642,7 @@ public class HRegion implements HeapSize { // , Writable{
* the wal. This method is then invoked to rollback the memstore.
*/
private void rollbackMemstore(BatchOperationInProgress<Mutation> batchOp,
Map<byte[], List<? extends Cell>>[] familyMaps,
Map<byte[], List<Cell>>[] familyMaps,
int start, int end) {
int kvsRolledback = 0;
for (int i = start; i < end; i++) {
@ -2653,10 +2653,10 @@ public class HRegion implements HeapSize { // , Writable{
}
// Rollback all the kvs for this row.
Map<byte[], List<? extends Cell>> familyMap = familyMaps[i];
for (Map.Entry<byte[], List<? extends Cell>> e : familyMap.entrySet()) {
Map<byte[], List<Cell>> familyMap = familyMaps[i];
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
byte[] family = e.getKey();
List<? extends Cell> cells = e.getValue();
List<Cell> cells = e.getValue();
// Remove those keys from the memstore that matches our
// key's (row, cf, cq, timestamp, memstoreTS). The interesting part is
@ -2688,7 +2688,7 @@ public class HRegion implements HeapSize { // , Writable{
* failure and replay
*/
private void removeNonExistentColumnFamilyForReplay(
final Map<byte[], List<? extends Cell>> familyMap) {
final Map<byte[], List<Cell>> familyMap) {
List<byte[]> nonExistentList = null;
for (byte[] family : familyMap.keySet()) {
if (!this.htableDescriptor.hasFamily(family)) {
@ -2707,13 +2707,13 @@ public class HRegion implements HeapSize { // , Writable{
}
}
void checkTimestamps(final Map<byte[], List<? extends Cell>> familyMap,
void checkTimestamps(final Map<byte[], List<Cell>> familyMap,
long now) throws FailedSanityCheckException {
if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
return;
}
long maxTs = now + timestampSlop;
for (List<? extends Cell> kvs : familyMap.values()) {
for (List<Cell> kvs : familyMap.values()) {
for (Cell cell : kvs) {
// see if the user-side TS is out of range. latest = server-side
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
@ -2731,9 +2731,9 @@ public class HRegion implements HeapSize { // , Writable{
* @param familyMap map of family->edits
* @param walEdit the destination entry to append into
*/
private void addFamilyMapToWALEdit(Map<byte[], List<? extends Cell>> familyMap,
private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
WALEdit walEdit) {
for (List<? extends Cell> edits : familyMap.values()) {
for (List<Cell> edits : familyMap.values()) {
for (Cell cell : edits) {
walEdit.add(KeyValueUtil.ensureKeyValue(cell));
}
@ -4174,7 +4174,7 @@ public class HRegion implements HeapSize { // , Writable{
// The row key is the region name
byte[] row = r.getRegionName();
final long now = EnvironmentEdgeManager.currentTimeMillis();
final List<KeyValue> cells = new ArrayList<KeyValue>(2);
final List<Cell> cells = new ArrayList<Cell>(2);
cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER, now,
r.getRegionInfo().toByteArray()));
@ -4645,8 +4645,8 @@ public class HRegion implements HeapSize { // , Writable{
Durability durability = getEffectiveDurability(append.getDurability());
boolean writeToWAL = durability != Durability.SKIP_WAL;
WALEdit walEdits = null;
List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
List<Cell> allKVs = new ArrayList<Cell>(append.size());
Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
long size = 0;
long txid = 0;
@ -4669,12 +4669,12 @@ public class HRegion implements HeapSize { // , Writable{
try {
long now = EnvironmentEdgeManager.currentTimeMillis();
// Process each family
for (Map.Entry<byte[], List<? extends Cell>> family : append.getFamilyCellMap().entrySet()) {
for (Map.Entry<byte[], List<Cell>> family : append.getFamilyCellMap().entrySet()) {
Store store = stores.get(family.getKey());
List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
Collections.sort((List<KeyValue>)family.getValue(), store.getComparator());
Collections.sort(family.getValue(), store.getComparator());
// Get previous values for all columns in this family
Get get = new Get(row);
for (Cell cell : family.getValue()) {
@ -4759,7 +4759,7 @@ public class HRegion implements HeapSize { // , Writable{
}
//Actually write to Memstore now
for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
for (Map.Entry<Store, List<Cell>> entry : tempMemstore.entrySet()) {
Store store = entry.getKey();
if (store.getFamily().getMaxVersions() == 1) {
// upsert if VERSIONS for this CF == 1
@ -4820,8 +4820,8 @@ public class HRegion implements HeapSize { // , Writable{
Durability durability = getEffectiveDurability(increment.getDurability());
boolean writeToWAL = durability != Durability.SKIP_WAL;
WALEdit walEdits = null;
List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.size());
Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
List<Cell> allKVs = new ArrayList<Cell>(increment.size());
Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
long size = 0;
long txid = 0;
@ -4843,11 +4843,11 @@ public class HRegion implements HeapSize { // , Writable{
try {
long now = EnvironmentEdgeManager.currentTimeMillis();
// Process each family
for (Map.Entry<byte [], List<? extends Cell>> family:
for (Map.Entry<byte [], List<Cell>> family:
increment.getFamilyCellMap().entrySet()) {
Store store = stores.get(family.getKey());
List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
// Get previous values for all columns in this family
Get get = new Get(row);
@ -4908,7 +4908,7 @@ public class HRegion implements HeapSize { // , Writable{
recordMutationWithoutWal(increment.getFamilyCellMap());
}
//Actually write to Memstore now
for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
for (Map.Entry<Store, List<Cell>> entry : tempMemstore.entrySet()) {
Store store = entry.getKey();
if (store.getFamily().getMaxVersions() == 1) {
// upsert if VERSIONS for this CF == 1
@ -5357,7 +5357,7 @@ public class HRegion implements HeapSize { // , Writable{
* Update counters for numer of puts without wal and the size of possible data loss.
* These information are exposed by the region server metrics.
*/
private void recordMutationWithoutWal(final Map<byte [], List<? extends Cell>> familyMap) {
private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
numMutationsWithoutWAL.increment();
if (numMutationsWithoutWAL.get() <= 1) {
LOG.info("writing data to region " + this +
@ -5365,7 +5365,7 @@ public class HRegion implements HeapSize { // , Writable{
}
long mutationSize = 0;
for (List<? extends Cell> cells: familyMap.values()) {
for (List<Cell> cells: familyMap.values()) {
for (Cell cell : cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
mutationSize += kv.getKeyLength() + kv.getValueLength();

View File

@ -1766,7 +1766,7 @@ public class HStore implements Store {
}
@Override
public long upsert(Iterable<? extends Cell> cells, long readpoint) throws IOException {
public long upsert(Iterable<Cell> cells, long readpoint) throws IOException {
this.lock.readLock().lock();
try {
return this.memstore.upsert(cells, readpoint);

View File

@ -555,7 +555,7 @@ public class MemStore implements HeapSize {
* @param readpoint readpoint below which we can safely remove duplicate KVs
* @return change in memstore size
*/
public long upsert(Iterable<? extends Cell> cells, long readpoint) {
public long upsert(Iterable<Cell> cells, long readpoint) {
this.lock.readLock().lock();
try {
long size = 0;

View File

@ -73,7 +73,7 @@ MultiRowMutationProcessorResponse> {
// Check mutations and apply edits to a single WALEdit
for (Mutation m : mutations) {
if (m instanceof Put) {
Map<byte[], List<? extends Cell>> familyMap = m.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = m.getFamilyCellMap();
region.checkFamilies(familyMap.keySet());
region.checkTimestamps(familyMap, now);
region.updateKVTimestamps(familyMap.values(), byteNow);
@ -86,7 +86,7 @@ MultiRowMutationProcessorResponse> {
"Action must be Put or Delete. But was: "
+ m.getClass().getName());
}
for (List<? extends Cell> cells: m.getFamilyCellMap().values()) {
for (List<Cell> cells: m.getFamilyCellMap().values()) {
boolean writeToWAL = m.getDurability() != Durability.SKIP_WAL;
for (Cell cell : cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);

View File

@ -113,7 +113,7 @@ public interface Store extends HeapSize, StoreConfigInformation {
* @return memstore size delta
* @throws IOException
*/
long upsert(Iterable<? extends Cell> cells, long readpoint) throws IOException;
long upsert(Iterable<Cell> cells, long readpoint) throws IOException;
/**
* Adds a value to the memstore

View File

@ -184,7 +184,7 @@ public class RemoteHTable implements HTableInterface {
protected CellSetModel buildModelFromPut(Put put) {
RowModel row = new RowModel(put.getRow());
long ts = put.getTimeStamp();
for (List<? extends Cell> cells: put.getFamilyCellMap().values()) {
for (List<Cell> cells: put.getFamilyCellMap().values()) {
for (Cell cell: cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(),
@ -404,7 +404,7 @@ public class RemoteHTable implements HTableInterface {
cells = new ArrayList<Cell>();
map.put(row, cells);
}
for (List<? extends Cell> l: put.getFamilyCellMap().values()) {
for (List<Cell> l: put.getFamilyCellMap().values()) {
cells.addAll(l);
}
}

View File

@ -160,11 +160,11 @@ public class AccessController extends BaseRegionObserver
* table updates.
*/
void updateACL(RegionCoprocessorEnvironment e,
final Map<byte[], List<? extends Cell>> familyMap) {
final Map<byte[], List<Cell>> familyMap) {
Set<byte[]> entries =
new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
for (Map.Entry<byte[], List<? extends Cell>> f : familyMap.entrySet()) {
List<? extends Cell> cells = f.getValue();
for (Map.Entry<byte[], List<Cell>> f : familyMap.entrySet()) {
List<Cell> cells = f.getValue();
for (Cell cell: cells) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
if (Bytes.equals(kv.getBuffer(), kv.getFamilyOffset(),
@ -1070,7 +1070,7 @@ public class AccessController extends BaseRegionObserver
throws IOException {
// Create a map of family to qualifiers.
Map<byte[], Set<byte[]>> familyMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
for (Map.Entry<byte [], List<? extends Cell>> entry: increment.getFamilyCellMap().entrySet()) {
for (Map.Entry<byte [], List<Cell>> entry: increment.getFamilyCellMap().entrySet()) {
Set<byte[]> qualifiers = Sets.newTreeSet(Bytes.BYTES_COMPARATOR);
for (Cell cell: entry.getValue()) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);

View File

@ -287,7 +287,7 @@ public class ThriftUtilities {
}
// Map<family, List<KeyValue>>
for (Map.Entry<byte[], List<? extends org.apache.hadoop.hbase.Cell>> familyEntry:
for (Map.Entry<byte[], List<org.apache.hadoop.hbase.Cell>> familyEntry:
in.getFamilyCellMap().entrySet()) {
TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) {

View File

@ -315,14 +315,14 @@ public class SimpleRegionObserver extends BaseRegionObserver {
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
final Put put, final WALEdit edit,
final Durability durability) throws IOException {
Map<byte[], List<? extends Cell>> familyMap = put.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = put.getFamilyCellMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
assertNotNull(e.getRegion());
assertNotNull(familyMap);
if (e.getRegion().getTableDesc().getTableName().equals(
TestRegionObserverInterface.TEST_TABLE)) {
List<? extends Cell> cells = familyMap.get(TestRegionObserverInterface.A);
List<Cell> cells = familyMap.get(TestRegionObserverInterface.A);
assertNotNull(cells);
assertNotNull(cells.get(0));
KeyValue kv = (KeyValue)cells.get(0);
@ -348,12 +348,12 @@ public class SimpleRegionObserver extends BaseRegionObserver {
public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c,
final Put put, final WALEdit edit,
final Durability durability) throws IOException {
Map<byte[], List<? extends Cell>> familyMap = put.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = put.getFamilyCellMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
assertNotNull(e.getRegion());
assertNotNull(familyMap);
List<? extends Cell> cells = familyMap.get(TestRegionObserverInterface.A);
List<Cell> cells = familyMap.get(TestRegionObserverInterface.A);
if (e.getRegion().getTableDesc().getTableName().equals(
TestRegionObserverInterface.TEST_TABLE)) {
assertNotNull(cells);
@ -381,7 +381,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
final Delete delete, final WALEdit edit,
final Durability durability) throws IOException {
Map<byte[], List<? extends Cell>> familyMap = delete.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = delete.getFamilyCellMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
assertNotNull(e.getRegion());
@ -395,7 +395,7 @@ public class SimpleRegionObserver extends BaseRegionObserver {
public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
final Delete delete, final WALEdit edit,
final Durability durability) throws IOException {
Map<byte[], List<? extends Cell>> familyMap = delete.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = delete.getFamilyCellMap();
RegionCoprocessorEnvironment e = c.getEnvironment();
assertNotNull(e);
assertNotNull(e.getRegion());

View File

@ -208,7 +208,7 @@ public class TestRegionObserverBypass {
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e,
final Put put, final WALEdit edit, final Durability durability)
throws IOException {
Map<byte[], List<? extends Cell>> familyMap = put.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = put.getFamilyCellMap();
if (familyMap.containsKey(test)) {
e.bypass();
}

View File

@ -157,7 +157,7 @@ public class TestWALObserver {
// Use a Put to create familyMap.
Put p = creatPutWith2Families(TEST_ROW);
Map<byte[], List<? extends Cell>> familyMap = p.getFamilyCellMap();
Map<byte[], List<Cell>> familyMap = p.getFamilyCellMap();
WALEdit edit = new WALEdit();
addFamilyMapToWALEdit(familyMap, edit);
@ -342,9 +342,9 @@ public class TestWALObserver {
* @param walEdit
* the destination entry to append into
*/
private void addFamilyMapToWALEdit(Map<byte[], List<? extends Cell>> familyMap,
private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
WALEdit walEdit) {
for (List<? extends Cell> edits : familyMap.values()) {
for (List<Cell> edits : familyMap.values()) {
for (Cell cell : edits) {
// KeyValue v1 expectation. Cast for now until we go all Cell all the time. TODO.
walEdit.add((KeyValue)cell);

View File

@ -40,16 +40,16 @@ public class TestReplicationProtobuf {
*/
@Test
public void testGetCellScanner() throws IOException {
List<KeyValue> a = new ArrayList<KeyValue>();
List<Cell> a = new ArrayList<Cell>();
KeyValue akv = new KeyValue(Bytes.toBytes("a"), -1L);
a.add(akv);
// Add a few just to make it less regular.
a.add(new KeyValue(Bytes.toBytes("aa"), -1L));
a.add(new KeyValue(Bytes.toBytes("aaa"), -1L));
List<KeyValue> b = new ArrayList<KeyValue>();
List<Cell> b = new ArrayList<Cell>();
KeyValue bkv = new KeyValue(Bytes.toBytes("b"), -1L);
a.add(bkv);
List<KeyValue> c = new ArrayList<KeyValue>();
List<Cell> c = new ArrayList<Cell>();
KeyValue ckv = new KeyValue(Bytes.toBytes("c"), -1L);
c.add(ckv);
List<List<? extends Cell>> all = new ArrayList<List<? extends Cell>>();

View File

@ -1221,15 +1221,15 @@ public class TestHRegion extends HBaseTestCase {
String method = this.getName();
this.region = initHRegion(tableName, method, conf, fam1, fam2, fam3);
try {
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
kvs.add(new KeyValue(row1, fam4, null, null));
//testing existing family
byte [] family = fam2;
try {
NavigableMap<byte[], List<? extends Cell>> deleteMap =
new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
NavigableMap<byte[], List<Cell>> deleteMap =
new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
deleteMap.put(family, kvs);
region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, Durability.SYNC_WAL);
} catch (Exception e) {
@ -1240,8 +1240,8 @@ public class TestHRegion extends HBaseTestCase {
boolean ok = false;
family = fam4;
try {
NavigableMap<byte[], List<? extends Cell>> deleteMap =
new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
NavigableMap<byte[], List<Cell>> deleteMap =
new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
deleteMap.put(family, kvs);
region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, Durability.SYNC_WAL);
} catch (Exception e) {
@ -1563,13 +1563,13 @@ public class TestHRegion extends HBaseTestCase {
this.region = initHRegion(tableName, method, conf, fam1);
try {
//Building checkerList
List<KeyValue> kvs = new ArrayList<KeyValue>();
List<Cell> kvs = new ArrayList<Cell>();
kvs.add(new KeyValue(row1, fam1, col1, null));
kvs.add(new KeyValue(row1, fam1, col2, null));
kvs.add(new KeyValue(row1, fam1, col3, null));
NavigableMap<byte[], List<? extends Cell>> deleteMap =
new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
NavigableMap<byte[], List<Cell>> deleteMap =
new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
deleteMap.put(fam1, kvs);
region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, Durability.SYNC_WAL);

View File

@ -327,9 +327,9 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
return put;
}
private void addFamilyMapToWALEdit(Map<byte[], List<? extends Cell>> familyMap,
private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
WALEdit walEdit) {
for (List<? extends Cell> edits : familyMap.values()) {
for (List<Cell> edits : familyMap.values()) {
for (Cell cell : edits) {
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
walEdit.add(kv);