You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/02/26 05:33:37 UTC

svn commit: r1450017 [2/2] - in /hbase/trunk: 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-common/src/main/java/org/apa...

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java Tue Feb 26 04:33:36 2013
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProcessorProtos.MultiRowMutationProcessorResponse;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.Cell;
 
 /**
  * A <code>MultiRowProcessor</code> that performs multiple puts and deletes.
@@ -70,7 +72,7 @@ MultiRowMutationProcessorResponse> {
     // Check mutations and apply edits to a single WALEdit
     for (Mutation m : mutations) {
       if (m instanceof Put) {
-        Map<byte[], List<KeyValue>> familyMap = m.getFamilyMap();
+        Map<byte[], List<? extends Cell>> familyMap = m.getFamilyMap();
         region.checkFamilies(familyMap.keySet());
         region.checkTimestamps(familyMap, now);
         region.updateKVTimestamps(familyMap.values(), byteNow);
@@ -83,9 +85,10 @@ MultiRowMutationProcessorResponse> {
             "Action must be Put or Delete. But was: "
             + m.getClass().getName());
       }
-      for (List<KeyValue> edits : m.getFamilyMap().values()) {
+      for (List<? extends Cell> cells: m.getFamilyMap().values()) {
         boolean writeToWAL = m.getWriteToWAL();
-        for (KeyValue kv : edits) {
+        for (Cell cell : cells) {
+          KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
           mutationKvs.add(kv);
           if (writeToWAL) {
             walEdit.add(kv);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Feb 26 04:33:36 2013
@@ -36,8 +36,7 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-
-import com.google.common.collect.ImmutableList;
+import org.apache.hbase.Cell;
 
 /**
  * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or
@@ -102,12 +101,12 @@ public interface Store extends HeapSize,
    * <p>
    * This operation is atomic on each KeyValue (row/family/qualifier) but not necessarily atomic
    * across all of them.
-   * @param kvs
+   * @param cells
    * @param readpoint readpoint below which we can safely remove duplicate KVs 
    * @return memstore size delta
    * @throws IOException
    */
-  public long upsert(Iterable<KeyValue> kvs, long readpoint) throws IOException;
+  public long upsert(Iterable<? extends Cell> cells, long readpoint) throws IOException;
 
   /**
    * Adds a value to the memstore

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Tue Feb 26 04:33:36 2013
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HBaseConf
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -63,6 +64,7 @@ import org.apache.hadoop.hbase.rest.mode
 import org.apache.hadoop.hbase.rest.model.ScannerModel;
 import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.Cell;
 
 /**
  * HTable interface to remote tables accessed via REST gateway
@@ -183,8 +185,9 @@ public class RemoteHTable implements HTa
   protected CellSetModel buildModelFromPut(Put put) {
     RowModel row = new RowModel(put.getRow());
     long ts = put.getTimeStamp();
-    for (List<KeyValue> kvs: put.getFamilyMap().values()) {
-      for (KeyValue kv: kvs) {
+    for (List<? extends Cell> cells: put.getFamilyMap().values()) {
+      for (Cell cell: cells) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(),
           ts != HConstants.LATEST_TIMESTAMP ? ts : kv.getTimestamp(),
           kv.getValue()));
@@ -388,25 +391,26 @@ public class RemoteHTable implements HTa
     // ignores the row specification in the URI
 
     // separate puts by row
-    TreeMap<byte[],List<KeyValue>> map =
-      new TreeMap<byte[],List<KeyValue>>(Bytes.BYTES_COMPARATOR);
+    TreeMap<byte[],List<Cell>> map =
+      new TreeMap<byte[],List<Cell>>(Bytes.BYTES_COMPARATOR);
     for (Put put: puts) {
       byte[] row = put.getRow();
-      List<KeyValue> kvs = map.get(row);
-      if (kvs == null) {
-        kvs = new ArrayList<KeyValue>();
-        map.put(row, kvs);
+      List<Cell> cells = map.get(row);
+      if (cells == null) {
+        cells = new ArrayList<Cell>();
+        map.put(row, cells);
       }
-      for (List<KeyValue> l: put.getFamilyMap().values()) {
-        kvs.addAll(l);
+      for (List<? extends Cell> l: put.getFamilyMap().values()) {
+        cells.addAll(l);
       }
     }
 
     // build the cell set
     CellSetModel model = new CellSetModel();
-    for (Map.Entry<byte[], List<KeyValue>> e: map.entrySet()) {
+    for (Map.Entry<byte[], List<Cell>> e: map.entrySet()) {
       RowModel row = new RowModel(e.getKey());
-      for (KeyValue kv: e.getValue()) {
+      for (Cell cell: e.getValue()) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         row.addCell(new CellModel(kv));
       }
       model.addRow(row);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java Tue Feb 26 04:33:36 2013
@@ -20,6 +20,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
@@ -69,6 +71,7 @@ import org.apache.hadoop.hbase.security.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hbase.Cell;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ListMultimap;
@@ -154,11 +157,12 @@ public class AccessController extends Ba
    * table updates.
    */
   void updateACL(RegionCoprocessorEnvironment e,
-      final Map<byte[], List<KeyValue>> familyMap) {
+      final Map<byte[], List<? extends Cell>> familyMap) {
     Set<byte[]> tableSet = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
-    for (Map.Entry<byte[], List<KeyValue>> f : familyMap.entrySet()) {
-      List<KeyValue> kvs = f.getValue();
-      for (KeyValue kv: kvs) {
+    for (Map.Entry<byte[], List<? extends Cell>> f : familyMap.entrySet()) {
+      List<? extends Cell> cells = f.getValue();
+      for (Cell cell: cells) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         if (Bytes.equals(kv.getBuffer(), kv.getFamilyOffset(),
             kv.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
             AccessControlLists.ACL_LIST_FAMILY.length)) {
@@ -964,9 +968,15 @@ public class AccessController extends Ba
   public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Increment increment)
       throws IOException {
+    // Create a map of family to qualifiers.
     Map<byte[], Set<byte[]>> familyMap = Maps.newHashMap();
-    for (Map.Entry<byte[], ? extends Map<byte[], Long>> entry : increment.getFamilyMap().entrySet()) {
-      familyMap.put(entry.getKey(), entry.getValue().keySet());
+    for (Map.Entry<byte [], List<? extends Cell>> entry: increment.getFamilyMap().entrySet()) {
+      Set<byte []> qualifiers = new HashSet<byte []>(entry.getValue().size());
+      for (Cell cell: entry.getValue()) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
+        qualifiers.add(kv.getQualifier());
+      }
+      familyMap.put(entry.getKey(), qualifiers);
     }
     requirePermission("increment", Permission.Action.WRITE, c.getEnvironment(), familyMap);
     return null;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java Tue Feb 26 04:33:36 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.thrift2;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.thrift2.generated.*;
 
@@ -37,13 +38,13 @@ public class ThriftUtilities {
 
   /**
    * Creates a {@link Get} (HBase) from a {@link TGet} (Thrift).
-   * 
+   *
    * This ignores any timestamps set on {@link TColumn} objects.
-   * 
+   *
    * @param in the <code>TGet</code> to convert
-   * 
+   *
    * @return <code>Get</code> object
-   * 
+   *
    * @throws IOException if an invalid time range or max version parameter is given
    */
   public static Get getFromThrift(TGet in) throws IOException {
@@ -77,11 +78,11 @@ public class ThriftUtilities {
 
   /**
    * Converts multiple {@link TGet}s (Thrift) into a list of {@link Get}s (HBase).
-   * 
+   *
    * @param in list of <code>TGet</code>s to convert
-   * 
+   *
    * @return list of <code>Get</code> objects
-   * 
+   *
    * @throws IOException if an invalid time range or max version parameter is given
    * @see #getFromThrift(TGet)
    */
@@ -95,9 +96,9 @@ public class ThriftUtilities {
 
   /**
    * Creates a {@link TResult} (Thrift) from a {@link Result} (HBase).
-   * 
+   *
    * @param in the <code>Result</code> to convert
-   * 
+   *
    * @return converted result, returns an empty result if the input is <code>null</code>
    */
   public static TResult resultFromHBase(Result in) {
@@ -122,11 +123,11 @@ public class ThriftUtilities {
 
   /**
    * Converts multiple {@link Result}s (HBase) into a list of {@link TResult}s (Thrift).
-   * 
+   *
    * @param in array of <code>Result</code>s to convert
-   * 
+   *
    * @return list of converted <code>TResult</code>s
-   * 
+   *
    * @see #resultFromHBase(Result)
    */
   public static List<TResult> resultsFromHBase(Result[] in) {
@@ -139,9 +140,9 @@ public class ThriftUtilities {
 
   /**
    * Creates a {@link Put} (HBase) from a {@link TPut} (Thrift)
-   * 
+   *
    * @param in the <code>TPut</code> to convert
-   * 
+   *
    * @return converted <code>Put</code>
    */
   public static Put putFromThrift(TPut in) {
@@ -169,11 +170,11 @@ public class ThriftUtilities {
 
   /**
    * Converts multiple {@link TPut}s (Thrift) into a list of {@link Put}s (HBase).
-   * 
+   *
    * @param in list of <code>TPut</code>s to convert
-   * 
+   *
    * @return list of converted <code>Put</code>s
-   * 
+   *
    * @see #putFromThrift(TPut)
    */
   public static List<Put> putsFromThrift(List<TPut> in) {
@@ -186,9 +187,9 @@ public class ThriftUtilities {
 
   /**
    * Creates a {@link Delete} (HBase) from a {@link TDelete} (Thrift).
-   * 
+   *
    * @param in the <code>TDelete</code> to convert
-   * 
+   *
    * @return converted <code>Delete</code>
    */
   public static Delete deleteFromThrift(TDelete in) {
@@ -233,11 +234,11 @@ public class ThriftUtilities {
 
   /**
    * Converts multiple {@link TDelete}s (Thrift) into a list of {@link Delete}s (HBase).
-   * 
+   *
    * @param in list of <code>TDelete</code>s to convert
-   * 
+   *
    * @return list of converted <code>Delete</code>s
-   * 
+   *
    * @see #deleteFromThrift(TDelete)
    */
 
@@ -259,12 +260,14 @@ public class ThriftUtilities {
     }
 
     // Map<family, List<KeyValue>>
-    for (Map.Entry<byte[], List<KeyValue>> familyEntry : in.getFamilyMap().entrySet()) {
+    for (Map.Entry<byte[], List<? extends org.apache.hbase.Cell>> familyEntry:
+        in.getFamilyMap().entrySet()) {
       TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
-      for (KeyValue keyValue : familyEntry.getValue()) {
-        byte[] family = keyValue.getFamily();
-        byte[] qualifier = keyValue.getQualifier();
-        long timestamp = keyValue.getTimestamp();
+      for (org.apache.hbase.Cell cell: familyEntry.getValue()) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
+        byte[] family = kv.getFamily();
+        byte[] qualifier = kv.getQualifier();
+        long timestamp = kv.getTimestamp();
         if (family != null) {
           column.setFamily(family);
         }
@@ -272,7 +275,7 @@ public class ThriftUtilities {
           column.setQualifier(qualifier);
         }
         if (timestamp != HConstants.LATEST_TIMESTAMP) {
-          column.setTimestamp(keyValue.getTimestamp());
+          column.setTimestamp(kv.getTimestamp());
         }
       }
       columns.add(column);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Tue Feb 26 04:33:36 2013
@@ -3634,7 +3634,8 @@ public class TestFromClientSide {
     assertEquals(put.size(), 1);
     assertEquals(put.getFamilyMap().get(CONTENTS_FAMILY).size(), 1);
 
-    KeyValue kv = put.getFamilyMap().get(CONTENTS_FAMILY).get(0);
+    // KeyValue v1 expectation.  Cast for now until we go all Cell all the time. TODO
+    KeyValue kv = (KeyValue)put.getFamilyMap().get(CONTENTS_FAMILY).get(0);
 
     assertTrue(Bytes.equals(kv.getFamily(), CONTENTS_FAMILY));
     // will it return null or an empty byte array?
@@ -4158,7 +4159,7 @@ public class TestFromClientSide {
     mrmBuilder.addMutationRequest(m2);
     MultiMutateRequest mrm = mrmBuilder.build();
     CoprocessorRpcChannel channel = t.coprocessorService(ROW);
-    MultiRowMutationService.BlockingInterface service = 
+    MultiRowMutationService.BlockingInterface service =
        MultiRowMutationService.newBlockingStub(channel);
     service.mutateRows(null, mrm);
     Get g = new Get(ROW);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java Tue Feb 26 04:33:36 2013
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hbase.Cell;
 
 /**
  * A sample region observer that tests the RegionObserver interface.
@@ -310,27 +311,30 @@ public class SimpleRegionObserver extend
   public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, 
       final Put put, final WALEdit edit,
       final boolean writeToWAL) throws IOException {
-    Map<byte[], List<KeyValue>> familyMap  = put.getFamilyMap();
+    Map<byte[], List<? extends Cell>> familyMap  = put.getFamilyMap();
     RegionCoprocessorEnvironment e = c.getEnvironment();
     assertNotNull(e);
     assertNotNull(e.getRegion());
     assertNotNull(familyMap);
     if (Arrays.equals(e.getRegion().getTableDesc().getName(),
         TestRegionObserverInterface.TEST_TABLE)) {
-      List<KeyValue> kvs = familyMap.get(TestRegionObserverInterface.A);
-      assertNotNull(kvs);
-      assertNotNull(kvs.get(0));
-      assertTrue(Bytes.equals(kvs.get(0).getQualifier(),
+      List<? extends Cell> cells = familyMap.get(TestRegionObserverInterface.A);
+      assertNotNull(cells);
+      assertNotNull(cells.get(0));
+      KeyValue kv = (KeyValue)cells.get(0);
+      assertTrue(Bytes.equals(kv.getQualifier(),
           TestRegionObserverInterface.A));
-      kvs = familyMap.get(TestRegionObserverInterface.B);
-      assertNotNull(kvs);
-      assertNotNull(kvs.get(0));
-      assertTrue(Bytes.equals(kvs.get(0).getQualifier(),
+      cells = familyMap.get(TestRegionObserverInterface.B);
+      assertNotNull(cells);
+      assertNotNull(cells.get(0));
+      kv = (KeyValue)cells.get(0);
+      assertTrue(Bytes.equals(kv.getQualifier(),
           TestRegionObserverInterface.B));
-      kvs = familyMap.get(TestRegionObserverInterface.C);
-      assertNotNull(kvs);
-      assertNotNull(kvs.get(0));
-      assertTrue(Bytes.equals(kvs.get(0).getQualifier(),
+      cells = familyMap.get(TestRegionObserverInterface.C);
+      assertNotNull(cells);
+      assertNotNull(cells.get(0));
+      kv = (KeyValue)cells.get(0);
+      assertTrue(Bytes.equals(kv.getQualifier(),
           TestRegionObserverInterface.C));
     }
     hadPrePut = true;
@@ -340,28 +344,31 @@ public class SimpleRegionObserver extend
   public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Put put, final WALEdit edit,
       final boolean writeToWAL) throws IOException {
-    Map<byte[], List<KeyValue>> familyMap  = put.getFamilyMap();
+    Map<byte[], List<? extends Cell>> familyMap  = put.getFamilyMap();
     RegionCoprocessorEnvironment e = c.getEnvironment();
     assertNotNull(e);
     assertNotNull(e.getRegion());
     assertNotNull(familyMap);
-    List<KeyValue> kvs = familyMap.get(TestRegionObserverInterface.A);
+    List<? extends Cell> cells = familyMap.get(TestRegionObserverInterface.A);
     if (Arrays.equals(e.getRegion().getTableDesc().getName(),
         TestRegionObserverInterface.TEST_TABLE)) {
-      assertNotNull(kvs);
-      assertNotNull(kvs.get(0));
-      assertTrue(Bytes.equals(kvs.get(0).getQualifier(),
-          TestRegionObserverInterface.A));
-      kvs = familyMap.get(TestRegionObserverInterface.B);
-      assertNotNull(kvs);
-      assertNotNull(kvs.get(0));
-      assertTrue(Bytes.equals(kvs.get(0).getQualifier(),
-          TestRegionObserverInterface.B));
-      kvs = familyMap.get(TestRegionObserverInterface.C);
-      assertNotNull(kvs);
-      assertNotNull(kvs.get(0));
-      assertTrue(Bytes.equals(kvs.get(0).getQualifier(),
-          TestRegionObserverInterface.C));
+      assertNotNull(cells);
+      assertNotNull(cells.get(0));
+      // KeyValue v1 expectation.  Cast for now until we go all Cell all the time. TODO
+      KeyValue kv = (KeyValue)cells.get(0);
+      assertTrue(Bytes.equals(kv.getQualifier(), TestRegionObserverInterface.A));
+      cells = familyMap.get(TestRegionObserverInterface.B);
+      assertNotNull(cells);
+      assertNotNull(cells.get(0));
+      // KeyValue v1 expectation.  Cast for now until we go all Cell all the time. TODO
+      kv = (KeyValue)cells.get(0);
+      assertTrue(Bytes.equals(kv.getQualifier(), TestRegionObserverInterface.B));
+      cells = familyMap.get(TestRegionObserverInterface.C);
+      assertNotNull(cells);
+      assertNotNull(cells.get(0));
+      // KeyValue v1 expectation.  Cast for now until we go all Cell all the time. TODO
+      kv = (KeyValue)cells.get(0);
+      assertTrue(Bytes.equals(kv.getQualifier(), TestRegionObserverInterface.C));
     }
     hadPostPut = true;
   }
@@ -370,7 +377,7 @@ public class SimpleRegionObserver extend
   public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c, 
       final Delete delete, final WALEdit edit,
       final boolean writeToWAL) throws IOException {
-    Map<byte[], List<KeyValue>> familyMap  = delete.getFamilyMap();
+    Map<byte[], List<? extends Cell>> familyMap  = delete.getFamilyMap();
     RegionCoprocessorEnvironment e = c.getEnvironment();
     assertNotNull(e);
     assertNotNull(e.getRegion());
@@ -384,7 +391,7 @@ public class SimpleRegionObserver extend
   public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c, 
       final Delete delete, final WALEdit edit,
       final boolean writeToWAL) throws IOException {
-    Map<byte[], List<KeyValue>> familyMap  = delete.getFamilyMap();
+    Map<byte[], List<? extends Cell>> familyMap  = delete.getFamilyMap();
     RegionCoprocessorEnvironment e = c.getEnvironment();
     assertNotNull(e);
     assertNotNull(e.getRegion());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java Tue Feb 26 04:33:36 2013
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
 import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
+import org.apache.hbase.Cell;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -206,12 +207,10 @@ public class TestRegionObserverBypass {
     public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e,
         final Put put, final WALEdit edit, final boolean writeToWAL)
         throws IOException {
-      Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
+      Map<byte[], List<? extends Cell>> familyMap = put.getFamilyMap();
       if (familyMap.containsKey(test)) {
         e.bypass();
       }
     }
   }
-
-}
-
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java Tue Feb 26 04:33:36 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.security.
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hbase.Cell;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -52,8 +53,9 @@ import java.util.Map;
 import static org.junit.Assert.*;
 
 /**
- * Tests invocation of the {@link org.apache.hadoop.hbase.coprocessor.MasterObserver}
- * interface hooks at all appropriate times during normal HMaster operations.
+ * Tests invocation of the
+ * {@link org.apache.hadoop.hbase.coprocessor.MasterObserver} interface hooks at
+ * all appropriate times during normal HMaster operations.
  */
 @Category(MediumTests.class)
 public class TestWALObserver {
@@ -62,17 +64,11 @@ public class TestWALObserver {
 
   private static byte[] TEST_TABLE = Bytes.toBytes("observedTable");
   private static byte[][] TEST_FAMILY = { Bytes.toBytes("fam1"),
-    Bytes.toBytes("fam2"),
-    Bytes.toBytes("fam3"),
-  };
+      Bytes.toBytes("fam2"), Bytes.toBytes("fam3"), };
   private static byte[][] TEST_QUALIFIER = { Bytes.toBytes("q1"),
-    Bytes.toBytes("q2"),
-    Bytes.toBytes("q3"),
-  };
+      Bytes.toBytes("q2"), Bytes.toBytes("q3"), };
   private static byte[][] TEST_VALUE = { Bytes.toBytes("v1"),
-    Bytes.toBytes("v2"),
-    Bytes.toBytes("v3"),
-  };
+      Bytes.toBytes("v2"), Bytes.toBytes("v3"), };
   private static byte[] TEST_ROW = Bytes.toBytes("testRow");
 
   private Configuration conf;
@@ -94,8 +90,8 @@ public class TestWALObserver {
     conf.setInt("dfs.client.block.recovery.retries", 2);
 
     TEST_UTIL.startMiniCluster(1);
-    Path hbaseRootDir =
-      TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
+    Path hbaseRootDir = TEST_UTIL.getDFSCluster().getFileSystem()
+        .makeQualified(new Path("/hbase"));
     LOG.info("hbase.rootdir=" + hbaseRootDir);
     conf.set(HConstants.HBASE_DIR, hbaseRootDir.toString());
   }
@@ -108,11 +104,12 @@ public class TestWALObserver {
   @Before
   public void setUp() throws Exception {
     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
-    //this.cluster = TEST_UTIL.getDFSCluster();
+    // this.cluster = TEST_UTIL.getDFSCluster();
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
     this.hbaseRootDir = new Path(conf.get(HConstants.HBASE_DIR));
     this.dir = new Path(this.hbaseRootDir, TestWALObserver.class.getName());
-    this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    this.oldLogDir = new Path(this.hbaseRootDir,
+        HConstants.HREGION_OLDLOGDIR_NAME);
     this.logDir = new Path(this.hbaseRootDir, HConstants.HREGION_LOGDIR_NAME);
     this.logName = HConstants.HREGION_LOGDIR_NAME;
 
@@ -127,21 +124,22 @@ public class TestWALObserver {
   }
 
   /**
-   * Test WAL write behavior with WALObserver. The coprocessor monitors
-   * a WALEdit written to WAL, and ignore, modify, and add KeyValue's for the
+   * Test WAL write behavior with WALObserver. The coprocessor monitors a
+   * WALEdit written to WAL, and ignore, modify, and add KeyValue's for the
    * WALEdit.
    */
   @Test
   public void testWALObserverWriteToWAL() throws Exception {
 
     HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
-    final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE));
+    final HTableDescriptor htd = createBasic3FamilyHTD(Bytes
+        .toString(TEST_TABLE));
 
     Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE));
     deleteDir(basedir);
     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
 
-    HLog log = HLogFactory.createHLog(this.fs, hbaseRootDir, 
+    HLog log = HLogFactory.createHLog(this.fs, hbaseRootDir,
         TestWALObserver.class.getName(), this.conf);
     SampleRegionWALObserver cp = getCoprocessor(log);
 
@@ -149,8 +147,7 @@ public class TestWALObserver {
     // TEST_FAMILY[1] value shall be changed.
     // TEST_FAMILY[2] shall be added to WALEdit, although it's not in the put.
     cp.setTestValues(TEST_TABLE, TEST_ROW, TEST_FAMILY[0], TEST_QUALIFIER[0],
-        TEST_FAMILY[1], TEST_QUALIFIER[1],
-        TEST_FAMILY[2], TEST_QUALIFIER[2]);
+        TEST_FAMILY[1], TEST_QUALIFIER[1], TEST_FAMILY[2], TEST_QUALIFIER[2]);
 
     assertFalse(cp.isPreWALWriteCalled());
     assertFalse(cp.isPostWALWriteCalled());
@@ -160,7 +157,7 @@ public class TestWALObserver {
     // Use a Put to create familyMap.
     Put p = creatPutWith2Families(TEST_ROW);
 
-    Map<byte [], List<KeyValue>> familyMap = p.getFamilyMap();
+    Map<byte[], List<? extends Cell>> familyMap = p.getFamilyMap();
     WALEdit edit = new WALEdit();
     addFamilyMapToWALEdit(familyMap, edit);
 
@@ -224,9 +221,12 @@ public class TestWALObserver {
     // WAL replay is handled at HRegion::replayRecoveredEdits(), which is
     // ultimately called by HRegion::initialize()
     byte[] tableName = Bytes.toBytes("testWALCoprocessorReplay");
-    final HTableDescriptor htd = getBasic3FamilyHTableDescriptor(Bytes.toString(tableName));
-    //final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
-    //final HRegionInfo hri1 = createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
+    final HTableDescriptor htd = getBasic3FamilyHTableDescriptor(Bytes
+        .toString(tableName));
+    // final HRegionInfo hri =
+    // createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
+    // final HRegionInfo hri1 =
+    // createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
     final HRegionInfo hri = new HRegionInfo(tableName, null, null);
 
     final Path basedir = new Path(this.hbaseRootDir, Bytes.toString(tableName));
@@ -235,19 +235,19 @@ public class TestWALObserver {
 
     final Configuration newConf = HBaseConfiguration.create(this.conf);
 
-    //HLog wal = new HLog(this.fs, this.dir, this.oldLogDir, this.conf);
+    // HLog wal = new HLog(this.fs, this.dir, this.oldLogDir, this.conf);
     HLog wal = createWAL(this.conf);
-    //Put p = creatPutWith2Families(TEST_ROW);
+    // Put p = creatPutWith2Families(TEST_ROW);
     WALEdit edit = new WALEdit();
     long now = EnvironmentEdgeManager.currentTimeMillis();
-    //addFamilyMapToWALEdit(p.getFamilyMap(), edit);
+    // addFamilyMapToWALEdit(p.getFamilyMap(), edit);
     final int countPerFamily = 1000;
-    //for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
-    for (HColumnDescriptor hcd: htd.getFamilies()) {
-          //addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
-          //EnvironmentEdgeManager.getDelegate(), wal);
+    // for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+    for (HColumnDescriptor hcd : htd.getFamilies()) {
+      // addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
+      // EnvironmentEdgeManager.getDelegate(), wal);
       addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
-      EnvironmentEdgeManager.getDelegate(), wal, htd);
+          EnvironmentEdgeManager.getDelegate(), wal, htd);
     }
     wal.append(hri, tableName, edit, now, htd);
     // sync to fs.
@@ -281,32 +281,34 @@ public class TestWALObserver {
   }
 
   /**
-   * Test to see CP loaded successfully or not. There is a duplication
-   * at TestHLog, but the purpose of that one is to see whether the loaded
-   * CP will impact existing HLog tests or not.
+   * Test to see CP loaded successfully or not. There is a duplication at
+   * TestHLog, but the purpose of that one is to see whether the loaded CP will
+   * impact existing HLog tests or not.
    */
   @Test
   public void testWALObserverLoaded() throws Exception {
     HLog log = HLogFactory.createHLog(fs, hbaseRootDir,
-                                      TestWALObserver.class.getName(), conf);
+        TestWALObserver.class.getName(), conf);
     assertNotNull(getCoprocessor(log));
   }
 
   private SampleRegionWALObserver getCoprocessor(HLog wal) throws Exception {
     WALCoprocessorHost host = wal.getCoprocessorHost();
-    Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class.getName());
-    return (SampleRegionWALObserver)c;
+    Coprocessor c = host.findCoprocessor(SampleRegionWALObserver.class
+        .getName());
+    return (SampleRegionWALObserver) c;
   }
 
   /*
    * Creates an HRI around an HTD that has <code>tableName</code> and three
    * column families named.
+   * 
    * @param tableName Name of table to use when we create HTableDescriptor.
    */
   private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
     HTableDescriptor htd = new HTableDescriptor(tableName);
 
-    for (int i = 0; i < TEST_FAMILY.length; i++ ) {
+    for (int i = 0; i < TEST_FAMILY.length; i++) {
       HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]);
       htd.addFamily(a);
     }
@@ -326,27 +328,30 @@ public class TestWALObserver {
 
   private Put creatPutWith2Families(byte[] row) throws IOException {
     Put p = new Put(row);
-    for (int i = 0; i < TEST_FAMILY.length-1; i++ ) {
-      p.add(TEST_FAMILY[i], TEST_QUALIFIER[i],
-          TEST_VALUE[i]);
+    for (int i = 0; i < TEST_FAMILY.length - 1; i++) {
+      p.add(TEST_FAMILY[i], TEST_QUALIFIER[i], TEST_VALUE[i]);
     }
     return p;
   }
 
   /**
    * Copied from HRegion.
-   *
-   * @param familyMap map of family->edits
-   * @param walEdit the destination entry to append into
+   * 
+   * @param familyMap
+   *          map of family->edits
+   * @param walEdit
+   *          the destination entry to append into
    */
-  private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
+  private void addFamilyMapToWALEdit(Map<byte[], List<? extends Cell>> familyMap,
       WALEdit walEdit) {
-    for (List<KeyValue> edits : familyMap.values()) {
-      for (KeyValue kv : edits) {
-        walEdit.add(kv);
+    for (List<? extends 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);
       }
     }
   }
+
   private Path runWALSplit(final Configuration c) throws IOException {
     FileSystem fs = FileSystem.get(c);
     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(c,
@@ -359,28 +364,31 @@ public class TestWALObserver {
     LOG.info("Split file=" + splits.get(0));
     return splits.get(0);
   }
+
   private HLog createWAL(final Configuration c) throws IOException {
     return HLogFactory.createHLog(FileSystem.get(c), hbaseRootDir, logName, c);
   }
-  private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
-      final byte [] rowName, final byte [] family,
-      final int count, EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
-  throws IOException {
+
+  private void addWALEdits(final byte[] tableName, final HRegionInfo hri,
+      final byte[] rowName, final byte[] family, final int count,
+      EnvironmentEdge ee, final HLog wal, final HTableDescriptor htd)
+      throws IOException {
     String familyStr = Bytes.toString(family);
     for (int j = 0; j < count; j++) {
       byte[] qualifierBytes = Bytes.toBytes(Integer.toString(j));
       byte[] columnBytes = Bytes.toBytes(familyStr + ":" + Integer.toString(j));
       WALEdit edit = new WALEdit();
-      edit.add(new KeyValue(rowName, family, qualifierBytes,
-        ee.currentTimeMillis(), columnBytes));
+      edit.add(new KeyValue(rowName, family, qualifierBytes, ee
+          .currentTimeMillis(), columnBytes));
       wal.append(hri, tableName, edit, ee.currentTimeMillis(), htd);
     }
   }
+
   private HTableDescriptor getBasic3FamilyHTableDescriptor(
       final String tableName) {
     HTableDescriptor htd = new HTableDescriptor(tableName);
 
-    for (int i = 0; i < TEST_FAMILY.length; i++ ) {
+    for (int i = 0; i < TEST_FAMILY.length; i++) {
       HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]);
       htd.addFamily(a);
     }
@@ -398,7 +406,4 @@ public class TestWALObserver {
     return htd;
   }
 
-
 }
-
-

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Tue Feb 26 04:33:36 2013
@@ -23,9 +23,9 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -93,6 +93,7 @@ import org.apache.hadoop.hbase.util.Incr
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hbase.Cell;
 import org.apache.hbase.CellComparator;
 import org.junit.Assert;
 import org.junit.Test;
@@ -1144,7 +1145,8 @@ public class TestHRegion extends HBaseTe
       //testing existing family
       byte [] family = fam2;
       try {
-        Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
+        NavigableMap<byte[], List<? extends Cell>> deleteMap =
+          new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
         deleteMap.put(family, kvs);
         region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
       } catch (Exception e) {
@@ -1155,7 +1157,8 @@ public class TestHRegion extends HBaseTe
       boolean ok = false;
       family = fam4;
       try {
-        Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
+        NavigableMap<byte[], List<? extends Cell>> deleteMap =
+          new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
         deleteMap.put(family, kvs);
         region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
       } catch (Exception e) {
@@ -1482,7 +1485,8 @@ public class TestHRegion extends HBaseTe
       kvs.add(new KeyValue(row1, fam1, col2, null));
       kvs.add(new KeyValue(row1, fam1, col3, null));
 
-      Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
+      NavigableMap<byte[], List<? extends Cell>> deleteMap =
+        new TreeMap<byte[], List<? extends Cell>>(Bytes.BYTES_COMPARATOR);
       deleteMap.put(fam1, kvs);
       region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java Tue Feb 26 04:33:36 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.Cell;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Iterables;
@@ -795,17 +796,17 @@ public class TestMemStore extends TestCa
   ////////////////////////////////////
   //Test for upsert with MSLAB
   ////////////////////////////////////
-  
+
   /**
    * Test a pathological pattern that shows why we can't currently
    * use the MSLAB for upsert workloads. This test inserts data
    * in the following pattern:
-   * 
+   *
    * - row0001 through row1000 (fills up one 2M Chunk)
    * - row0002 through row1001 (fills up another 2M chunk, leaves one reference
    *   to the first chunk
    * - row0003 through row1002 (another chunk, another dangling reference)
-   * 
+   *
    * This causes OOME pretty quickly if we use MSLAB for upsert
    * since each 2M chunk is held onto by a single reference.
    */
@@ -813,17 +814,17 @@ public class TestMemStore extends TestCa
     Configuration conf = HBaseConfiguration.create();
     conf.setBoolean(MemStore.USEMSLAB_KEY, true);
     memstore = new MemStore(conf, KeyValue.COMPARATOR);
-    
+
     int ROW_SIZE = 2048;
     byte[] qualifier = new byte[ROW_SIZE - 4];
-    
+
     MemoryMXBean bean = ManagementFactory.getMemoryMXBean();
     for (int i = 0; i < 3; i++) { System.gc(); }
     long usageBefore = bean.getHeapMemoryUsage().getUsed();
-    
+
     long size = 0;
     long ts=0;
-    
+
     for (int newValue = 0; newValue < 1000; newValue++) {
       for (int row = newValue; row < newValue + 1000; row++) {
         byte[] rowBytes = Bytes.toBytes(row);
@@ -834,10 +835,10 @@ public class TestMemStore extends TestCa
     for (int i = 0; i < 3; i++) { System.gc(); }
     long usageAfter = bean.getHeapMemoryUsage().getUsed();
     System.out.println("Memory used: " + (usageAfter - usageBefore)
-        + " (heapsize: " + memstore.heapSize() + 
+        + " (heapsize: " + memstore.heapSize() +
         " size: " + size + ")");
   }
-  
+
   //////////////////////////////////////////////////////////////////////////////
   // Helpers
   //////////////////////////////////////////////////////////////////////////////
@@ -855,19 +856,19 @@ public class TestMemStore extends TestCa
     Configuration conf = HBaseConfiguration.create();
     memstore = new MemStore(conf, KeyValue.COMPARATOR);
     long oldSize = memstore.size.get();
-    
-    List<KeyValue> l = new ArrayList<KeyValue>();
+
+    List<Cell> l = new ArrayList<Cell>();
     KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v");
     KeyValue kv2 = KeyValueTestUtil.create("r", "f", "q", 101, "v");
     KeyValue kv3 = KeyValueTestUtil.create("r", "f", "q", 102, "v");
 
     kv1.setMvccVersion(1); kv2.setMvccVersion(1);kv3.setMvccVersion(1);
     l.add(kv1); l.add(kv2); l.add(kv3);
-    
+
     this.memstore.upsert(l, 2);// readpoint is 2
     long newSize = this.memstore.size.get();
     assert(newSize > oldSize);
-    
+
     KeyValue kv4 = KeyValueTestUtil.create("r", "f", "q", 104, "v");
     kv4.setMvccVersion(1);
     l.clear(); l.add(kv4);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java Tue Feb 26 04:33:36 2013
@@ -18,37 +18,33 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import java.util.Map;
+import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
-import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.classification.InterfaceAudience;
-
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
-import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
-import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hbase.Cell;
 
 /**
  * This class runs performance benchmarks for {@link HLog}.
@@ -330,9 +326,11 @@ public final class HLogPerformanceEvalua
     return put;
   }
 
-  private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap, WALEdit walEdit) {
-    for (List<KeyValue> edits : familyMap.values()) {
-      for (KeyValue kv : edits) {
+  private void addFamilyMapToWALEdit(Map<byte[], List<? extends Cell>> familyMap,
+      WALEdit walEdit) {
+    for (List<? extends Cell> edits : familyMap.values()) {
+      for (Cell cell : edits) {
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         walEdit.add(kv);
       }
     }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java?rev=1450017&r1=1450016&r2=1450017&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java Tue Feb 26 04:33:36 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
@@ -55,13 +56,12 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.Cell;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -217,11 +217,13 @@ public class TestCoprocessorScanPolicy {
     public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c, final Put put,
         final WALEdit edit, final boolean writeToWAL) throws IOException {
       if (put.getAttribute("ttl") != null) {
-        KeyValue kv = put.getFamilyMap().values().iterator().next().get(0);
+        Cell cell = put.getFamilyMap().values().iterator().next().get(0);
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         ttls.put(Bytes.toString(kv.getQualifier()), Bytes.toLong(kv.getValue()));
         c.bypass();
       } else if (put.getAttribute("versions") != null) {
-        KeyValue kv = put.getFamilyMap().values().iterator().next().get(0);
+        Cell cell = put.getFamilyMap().values().iterator().next().get(0);
+        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
         versions.put(Bytes.toString(kv.getQualifier()), Bytes.toInt(kv.getValue()));
         c.bypass();
       }