You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/08/15 19:49:04 UTC

svn commit: r1514399 - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/a...

Author: jmhsieh
Date: Thu Aug 15 17:49:03 2013
New Revision: 1514399

URL: http://svn.apache.org/r1514399
Log:
HBASE-9164 Convert List<? extends Cell> anti pattern to List<Cell> pattern

This patch also starts a refactor of the KVComparator.

Modified:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java Thu Aug 15 17:49:03 2013
@@ -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;
   }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java Thu Aug 15 17:49:03 2013
@@ -161,13 +161,11 @@ public class Delete extends Mutation imp
         " 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 imp
    */
   @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 imp
    * @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 imp
    */
   @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 imp
    */
   @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;
   }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Thu Aug 15 17:49:03 2013
@@ -1328,7 +1328,7 @@ public class HTable implements HTableInt
       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);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java Thu Aug 15 17:49:03 2013
@@ -83,7 +83,7 @@ public class Increment extends Mutation 
   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 
       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 
     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 
    * @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 
     }
     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 {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java Thu Aug 15 17:49:03 2013
@@ -65,8 +65,8 @@ public abstract class Mutation extends O
   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 O
    * @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 O
     // 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 O
     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);
@@ -215,9 +215,9 @@ public abstract class Mutation extends O
   @Deprecated
   public Map<byte[], List<KeyValue>> getFamilyMap() {
     Map<byte[], List<KeyValue>> fm = new TreeMap();
-    for (Map.Entry<byte[], List<? extends Cell>> e : this.familyMap.entrySet()) {
+    for (Map.Entry<byte[], List<Cell>> e : this.familyMap.entrySet()) {
       byte[] family = e.getKey();
-      List<? extends Cell> cells = e.getValue();
+      List<Cell> cells = e.getValue();
       List<KeyValue> kvs = new ArrayList(cells.size());
       for (Cell c : cells) {
          KeyValue kv = KeyValueUtil.ensureKeyValue(c);
@@ -232,14 +232,14 @@ public abstract class Mutation extends O
    * 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;
@@ -303,7 +303,7 @@ public abstract class Mutation extends O
    */
   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;
@@ -328,7 +328,7 @@ public abstract class Mutation extends O
     // 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);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java Thu Aug 15 17:49:03 2013
@@ -92,8 +92,8 @@ public class Put extends Mutation implem
    */
   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 implem
    */
   @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 implem
   @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 implem
       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 implem
    */
   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;
     }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Thu Aug 15 17:49:03 2013
@@ -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()) {

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java Thu Aug 15 17:49:03 2013
@@ -215,11 +215,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

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Thu Aug 15 17:49:03 2013
@@ -98,12 +98,6 @@ public class KeyValue implements Cell, H
   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.
    *
    * Hopefully we can get rid of this, I added this here because it's replacing
@@ -871,7 +865,7 @@ public class KeyValue implements Cell, H
     // 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 +876,7 @@ public class KeyValue implements Cell, H
    */
   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 +1773,61 @@ public class KeyValue implements Cell, H
     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 +1836,7 @@ public class KeyValue implements Cell, H
    * 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 +1847,81 @@ public class KeyValue implements Cell, H
       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 +1936,7 @@ public class KeyValue implements Cell, H
       // Compare timestamps
       long ltimestamp = left.getTimestamp(lkeylength);
       long rtimestamp = right.getTimestamp(rkeylength);
-      return getRawComparator().compareTimestamps(ltimestamp, rtimestamp);
+      return KeyComparator.compareTimestamps(ltimestamp, rtimestamp);
     }
 
     /**
@@ -1964,7 +2082,6 @@ public class KeyValue implements Cell, H
     protected Object clone() throws CloneNotSupportedException {
       return new KVComparator();
     }
-
   }
 
   /**
@@ -2353,8 +2470,6 @@ public class KeyValue implements Cell, H
   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 +2796,7 @@ public class KeyValue implements Cell, H
         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

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutCombiner.java Thu Aug 15 17:49:03 2013
@@ -56,17 +56,17 @@ public class PutCombiner<K> extends Redu
     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();

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/PutSortReducer.java Thu Aug 15 17:49:03 2013
@@ -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);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java Thu Aug 15 17:49:03 2013
@@ -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

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Aug 15 17:49:03 2013
@@ -1799,7 +1799,7 @@ public class HRegion implements HeapSize
    * @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);
@@ -1815,12 +1815,12 @@ public class HRegion implements HeapSize
    * @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) {
@@ -2026,7 +2026,7 @@ public class HRegion implements HeapSize
     /** 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;
@@ -2043,7 +2043,7 @@ public class HRegion implements HeapSize
         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;
 
@@ -2506,8 +2506,8 @@ public class HRegion implements HeapSize
    * 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);
@@ -2599,10 +2599,10 @@ public class HRegion implements HeapSize
    * @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);
@@ -2623,7 +2623,7 @@ public class HRegion implements HeapSize
    * @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;
@@ -2634,9 +2634,9 @@ public class HRegion implements HeapSize
         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) {
@@ -2660,7 +2660,7 @@ public class HRegion implements HeapSize
    * 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++) {
@@ -2671,10 +2671,10 @@ public class HRegion implements HeapSize
       }
 
       // 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
@@ -2706,7 +2706,7 @@ public class HRegion implements HeapSize
    * 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)) {
@@ -2725,13 +2725,13 @@ public class HRegion implements HeapSize
     }
   }
 
-  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);
@@ -2749,9 +2749,9 @@ public class HRegion implements HeapSize
    * @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));
       }
@@ -4198,7 +4198,7 @@ public class HRegion implements HeapSize
     // 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()));
@@ -4669,8 +4669,8 @@ public class HRegion implements HeapSize
     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;
@@ -4693,12 +4693,12 @@ public class HRegion implements HeapSize
         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());
-
-            Collections.sort((List<KeyValue>)family.getValue(), store.getComparator());
+            List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
+  
+            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()) {
@@ -4783,7 +4783,7 @@ public class HRegion implements HeapSize
           }
 
           //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
@@ -4844,8 +4844,8 @@ public class HRegion implements HeapSize
     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;
@@ -4867,11 +4867,11 @@ public class HRegion implements HeapSize
         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);
@@ -4932,7 +4932,7 @@ public class HRegion implements HeapSize
             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
@@ -5381,7 +5381,7 @@ public class HRegion implements HeapSize
    * 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 +
@@ -5389,7 +5389,7 @@ public class HRegion implements HeapSize
     }
 
     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();

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java Thu Aug 15 17:49:03 2013
@@ -1774,7 +1774,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);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java Thu Aug 15 17:49:03 2013
@@ -555,7 +555,7 @@ public class MemStore implements HeapSiz
    * @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;

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java Thu Aug 15 17:49:03 2013
@@ -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);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Thu Aug 15 17:49:03 2013
@@ -113,7 +113,7 @@ public interface Store extends HeapSize,
    * @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

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Thu Aug 15 17:49:03 2013
@@ -184,7 +184,7 @@ public class RemoteHTable implements HTa
   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 HTa
         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);
       }
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java Thu Aug 15 17:49:03 2013
@@ -161,10 +161,11 @@ public class AccessController extends Ba
    * table updates.
    */
   void updateACL(RegionCoprocessorEnvironment e,
-      final Map<byte[], List<? extends 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();
+      final Map<byte[], List<Cell>> familyMap) {
+    Set<byte[]> entries =
+        new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
+    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 +1071,7 @@ public class AccessController extends Ba
       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);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java Thu Aug 15 17:49:03 2013
@@ -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()) {

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java Thu Aug 15 17:49:03 2013
@@ -315,14 +315,14 @@ public class SimpleRegionObserver extend
   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 extend
   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 extend
   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 extend
   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());

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java Thu Aug 15 17:49:03 2013
@@ -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();
       }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java Thu Aug 15 17:49:03 2013
@@ -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);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestReplicationProtobuf.java Thu Aug 15 17:49:03 2013
@@ -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>>();

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Thu Aug 15 17:49:03 2013
@@ -1221,15 +1221,15 @@ public class TestHRegion extends HBaseTe
     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 HBaseTe
       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 HBaseTe
     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);
 

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java?rev=1514399&r1=1514398&r2=1514399&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java Thu Aug 15 17:49:03 2013
@@ -327,9 +327,9 @@ public final class HLogPerformanceEvalua
     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);