You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2014/04/09 13:46:16 UTC

svn commit: r1585945 [2/3] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apach...

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

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java Wed Apr  9 11:46:14 2014
@@ -20,8 +20,8 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.FileNotFoundException;
-import java.io.InterruptedIOException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.Reference;
@@ -534,7 +535,7 @@ public class HRegionFileSystem {
     // If it is outside the range, return directly.
     if (top) {
       //check if larger than last key.
-      KeyValue splitKey = KeyValue.createFirstOnRow(splitRow);
+      KeyValue splitKey = KeyValueUtil.createFirstOnRow(splitRow);
       byte[] lastKey = f.createReader().getLastKey();      
       // If lastKey is null means storefile is empty.
       if (lastKey == null) return null;
@@ -544,7 +545,7 @@ public class HRegionFileSystem {
       }
     } else {
       //check if smaller than first key
-      KeyValue splitKey = KeyValue.createLastOnRow(splitRow);
+      KeyValue splitKey = KeyValueUtil.createLastOnRow(splitRow);
       byte[] firstKey = f.createReader().getFirstKey();
       // If firstKey is null means storefile is empty.
       if (firstKey == null) return null;

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

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

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonLazyKeyValueScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonLazyKeyValueScanner.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonLazyKeyValueScanner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonLazyKeyValueScanner.java Wed Apr  9 11:46:14 2014
@@ -23,7 +23,7 @@ import java.util.SortedSet;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Scan;
 
 /**
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.Sc
 public abstract class NonLazyKeyValueScanner implements KeyValueScanner {
 
   @Override
-  public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
+  public boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
       throws IOException {
     return doRealSeek(this, kv, forward);
   }
@@ -51,7 +51,7 @@ public abstract class NonLazyKeyValueSca
   }
 
   public static boolean doRealSeek(KeyValueScanner scanner,
-      KeyValue kv, boolean forward) throws IOException {
+      Cell kv, boolean forward) throws IOException {
     return forward ? scanner.reseek(kv) : scanner.seek(kv);
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonReversedNonLazyKeyValueScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonReversedNonLazyKeyValueScanner.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonReversedNonLazyKeyValueScanner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NonReversedNonLazyKeyValueScanner.java Wed Apr  9 11:46:14 2014
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 
 /**
  * A "non-reversed & non-lazy" scanner which does not support backward scanning
@@ -34,13 +34,13 @@ public abstract class NonReversedNonLazy
     NonLazyKeyValueScanner {
 
   @Override
-  public boolean backwardSeek(KeyValue key) throws IOException {
+  public boolean backwardSeek(Cell key) throws IOException {
     throw new NotImplementedException("backwardSeek must not be called on a "
         + "non-reversed scanner");
   }
 
   @Override
-  public boolean seekToPreviousRow(KeyValue key) throws IOException {
+  public boolean seekToPreviousRow(Cell key) throws IOException {
     throw new NotImplementedException("seekToPreviousRow must not be called on a "
         + "non-reversed scanner");
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedKeyValueHeap.java Wed Apr  9 11:46:14 2014
@@ -23,7 +23,8 @@ import java.util.List;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 
 /**
@@ -47,26 +48,26 @@ public class ReversedKeyValueHeap extend
   }
 
   @Override
-  public boolean seek(KeyValue seekKey) throws IOException {
+  public boolean seek(Cell seekKey) throws IOException {
     throw new IllegalStateException(
         "seek cannot be called on ReversedKeyValueHeap");
   }
 
   @Override
-  public boolean reseek(KeyValue seekKey) throws IOException {
+  public boolean reseek(Cell seekKey) throws IOException {
     throw new IllegalStateException(
         "reseek cannot be called on ReversedKeyValueHeap");
   }
 
   @Override
-  public boolean requestSeek(KeyValue key, boolean forward, boolean useBloom)
+  public boolean requestSeek(Cell key, boolean forward, boolean useBloom)
       throws IOException {
     throw new IllegalStateException(
         "requestSeek cannot be called on ReversedKeyValueHeap");
   }
 
   @Override
-  public boolean seekToPreviousRow(KeyValue seekKey) throws IOException {
+  public boolean seekToPreviousRow(Cell seekKey) throws IOException {
     if (current == null) {
       return false;
     }
@@ -75,7 +76,7 @@ public class ReversedKeyValueHeap extend
 
     KeyValueScanner scanner;
     while ((scanner = heap.poll()) != null) {
-      KeyValue topKey = scanner.peek();
+      Cell topKey = scanner.peek();
       if (comparator.getComparator().compareRows(topKey.getRowArray(),
           topKey.getRowOffset(), topKey.getRowLength(), seekKey.getRowArray(),
           seekKey.getRowOffset(), seekKey.getRowLength()) < 0) {
@@ -97,7 +98,7 @@ public class ReversedKeyValueHeap extend
   }
 
   @Override
-  public boolean backwardSeek(KeyValue seekKey) throws IOException {
+  public boolean backwardSeek(Cell seekKey) throws IOException {
     if (current == null) {
       return false;
     }
@@ -106,8 +107,8 @@ public class ReversedKeyValueHeap extend
 
     KeyValueScanner scanner;
     while ((scanner = heap.poll()) != null) {
-      KeyValue topKey = scanner.peek();
-      if ((comparator.getComparator().matchingRows(seekKey, topKey) && comparator
+      Cell topKey = scanner.peek();
+      if ((CellUtil.matchingRow(seekKey, topKey) && comparator
           .getComparator().compare(seekKey, topKey) <= 0)
           || comparator.getComparator().compareRows(seekKey, topKey) > 0) {
         heap.add(scanner);
@@ -124,12 +125,12 @@ public class ReversedKeyValueHeap extend
   }
 
   @Override
-  public KeyValue next() throws IOException {
+  public Cell next() throws IOException {
     if (this.current == null) {
       return null;
     }
-    KeyValue kvReturn = this.current.next();
-    KeyValue kvNext = this.current.peek();
+    Cell kvReturn = this.current.next();
+    Cell kvNext = this.current.peek();
     if (kvNext == null
         || this.comparator.kvComparator.compareRows(kvNext, kvReturn) > 0) {
       if (this.current.seekToPreviousRow(kvReturn)) {
@@ -180,7 +181,7 @@ public class ReversedKeyValueHeap extend
      * @param right
      * @return less than 0 if left is smaller, 0 if equal etc..
      */
-    public int compareRows(KeyValue left, KeyValue right) {
+    public int compareRows(Cell left, Cell right) {
       return super.kvComparator.compareRows(left, right);
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedRegionScannerImpl.java Wed Apr  9 11:46:14 2014
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 
@@ -68,7 +68,7 @@ class ReversedRegionScannerImpl extends 
     assert super.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read.";
     byte row[] = new byte[length];
     System.arraycopy(currentRow, offset, row, 0, length);
-    this.storeHeap.seekToPreviousRow(KeyValue.createFirstOnRow(row));
+    this.storeHeap.seekToPreviousRow(KeyValueUtil.createFirstOnRow(row));
     resetFilters();
     // Calling the hook in CP which allows it to do a fast forward
     if (this.region.getCoprocessorHost() != null) {

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java Wed Apr  9 11:46:14 2014
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
@@ -151,7 +152,7 @@ public class ScanQueryMatcher {
     this.rowComparator = scanInfo.getComparator();
     this.deletes =  new ScanDeleteTracker();
     this.stopRow = scan.getStopRow();
-    this.startKey = KeyValue.createFirstDeleteFamilyOnRow(scan.getStartRow(),
+    this.startKey = KeyValueUtil.createFirstDeleteFamilyOnRow(scan.getStartRow(),
         scanInfo.getFamily());
     this.filter = scan.getFilter();
     this.earliestPutTs = earliestPutTs;
@@ -535,12 +536,12 @@ public class ScanQueryMatcher {
   public KeyValue getKeyForNextColumn(KeyValue kv) {
     ColumnCount nextColumn = columns.getColumnHint();
     if (nextColumn == null) {
-      return KeyValue.createLastOnRow(
+      return KeyValueUtil.createLastOnRow(
           kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
           kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
           kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
     } else {
-      return KeyValue.createFirstOnRow(
+      return KeyValueUtil.createFirstOnRow(
           kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
           kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
           nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength());
@@ -548,7 +549,7 @@ public class ScanQueryMatcher {
   }
 
   public KeyValue getKeyForNextRow(KeyValue kv) {
-    return KeyValue.createLastOnRow(
+    return KeyValueUtil.createLastOnRow(
         kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
         null, 0, 0,
         null, 0, 0);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Wed Apr  9 11:46:14 2014
@@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -1311,11 +1311,11 @@ public class StoreFile {
           && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
         return true;
       }
-      KeyValue smallestScanKeyValue = scan.isReversed() ? KeyValue
-          .createFirstOnRow(scan.getStopRow()) : KeyValue.createFirstOnRow(scan
+      KeyValue smallestScanKeyValue = scan.isReversed() ? KeyValueUtil
+          .createFirstOnRow(scan.getStopRow()) : KeyValueUtil.createFirstOnRow(scan
           .getStartRow());
-      KeyValue largestScanKeyValue = scan.isReversed() ? KeyValue
-          .createLastOnRow(scan.getStartRow()) : KeyValue.createLastOnRow(scan
+      KeyValue largestScanKeyValue = scan.isReversed() ? KeyValueUtil
+          .createLastOnRow(scan.getStartRow()) : KeyValueUtil.createLastOnRow(scan
           .getStopRow());
       boolean nonOverLapping = (getComparator().compareFlatKey(
           this.getFirstKey(), largestScanKeyValue.getKey()) > 0 && !Bytes

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java Wed Apr  9 11:46:14 2014
@@ -29,8 +29,11 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
@@ -47,11 +50,11 @@ public class StoreFileScanner implements
   // the reader it comes from:
   private final StoreFile.Reader reader;
   private final HFileScanner hfs;
-  private KeyValue cur = null;
+  private Cell cur = null;
 
   private boolean realSeekDone;
   private boolean delayedReseek;
-  private KeyValue delayedSeekKV;
+  private Cell delayedSeekKV;
 
   private boolean enforceMVCC = false;
   private boolean hasMVCCInfo = false;
@@ -124,12 +127,12 @@ public class StoreFileScanner implements
     return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]";
   }
 
-  public KeyValue peek() {
+  public Cell peek() {
     return cur;
   }
 
-  public KeyValue next() throws IOException {
-    KeyValue retKey = cur;
+  public Cell next() throws IOException {
+    Cell retKey = cur;
 
     try {
       // only seek if we aren't at the end. cur == null implies 'end'.
@@ -145,7 +148,7 @@ public class StoreFileScanner implements
     return retKey;
   }
 
-  public boolean seek(KeyValue key) throws IOException {
+  public boolean seek(Cell key) throws IOException {
     if (seekCount != null) seekCount.incrementAndGet();
 
     try {
@@ -166,7 +169,7 @@ public class StoreFileScanner implements
     }
   }
 
-  public boolean reseek(KeyValue key) throws IOException {
+  public boolean reseek(Cell key) throws IOException {
     if (seekCount != null) seekCount.incrementAndGet();
 
     try {
@@ -190,7 +193,7 @@ public class StoreFileScanner implements
   protected boolean skipKVsNewerThanReadpoint() throws IOException {
     // We want to ignore all key-values that are newer than our current
     // readPoint
-    KeyValue startKV = cur;
+    Cell startKV = cur;
     while(enforceMVCC
         && cur != null
         && (cur.getMvccVersion() > readPt)) {
@@ -216,7 +219,7 @@ public class StoreFileScanner implements
     // not old enough during flush). Make sure that we set it correctly now,
     // so that the comparision order does not change.
     if (cur.getMvccVersion() <= readPt) {
-      cur.setMvccVersion(0);
+      KeyValueUtil.ensureKeyValue(cur).setMvccVersion(0);
     }
     return true;
   }
@@ -233,7 +236,7 @@ public class StoreFileScanner implements
    * @return false if not found or if k is after the end.
    * @throws IOException
    */
-  public static boolean seekAtOrAfter(HFileScanner s, KeyValue k)
+  public static boolean seekAtOrAfter(HFileScanner s, Cell k)
   throws IOException {
     int result = s.seekTo(k);
     if(result < 0) {
@@ -252,7 +255,7 @@ public class StoreFileScanner implements
     return true;
   }
 
-  static boolean reseekAtOrAfter(HFileScanner s, KeyValue k)
+  static boolean reseekAtOrAfter(HFileScanner s, Cell k)
   throws IOException {
     //This function is similar to seekAtOrAfter function
     int result = s.reseekTo(k);
@@ -294,7 +297,7 @@ public class StoreFileScanner implements
    * row/column and use OLDEST_TIMESTAMP in the seek key.
    */
   @Override
-  public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
+  public boolean requestSeek(Cell kv, boolean forward, boolean useBloom)
       throws IOException {
     if (kv.getFamilyLength() == 0) {
       useBloom = false;
@@ -308,7 +311,7 @@ public class StoreFileScanner implements
             kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
             kv.getQualifierOffset(), kv.getQualifierLength());
       } else if (this.matcher != null && !matcher.hasNullColumnInQuery() &&
-          (kv.isDeleteFamily() || kv.isDeleteFamilyVersion())) {
+          ((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) {
         // if there is no such delete family kv in the store file,
         // then no need to seek.
         haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(),
@@ -332,7 +335,7 @@ public class StoreFileScanner implements
         // a higher timestamp than the max timestamp in this file. We know that
         // the next point when we have to consider this file again is when we
         // pass the max timestamp of this file (with the same row/column).
-        cur = kv.createFirstOnRowColTS(maxTimestampInFile);
+        cur = KeyValueUtil.createFirstOnRowColTS(kv, maxTimestampInFile);
       } else {
         // This will be the case e.g. when we need to seek to the next
         // row/column, and we don't know exactly what they are, so we set the
@@ -350,7 +353,7 @@ public class StoreFileScanner implements
     // key/value and the store scanner will progress to the next column. This
     // is obviously not a "real real" seek, but unlike the fake KV earlier in
     // this method, we want this to be propagated to ScanQueryMatcher.
-    cur = kv.createLastOnRowCol();
+    cur = KeyValueUtil.createLastOnRowCol(kv);
 
     realSeekDone = true;
     return true;
@@ -402,18 +405,19 @@ public class StoreFileScanner implements
   }
 
   @Override
-  public boolean seekToPreviousRow(KeyValue key) throws IOException {
+  public boolean seekToPreviousRow(Cell key) throws IOException {
     try {
       try {
-        KeyValue seekKey = KeyValue.createFirstOnRow(key.getRow());
+        KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
+            key.getRowLength());
         if (seekCount != null) seekCount.incrementAndGet();
         if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(),
             seekKey.getKeyLength())) {
           close();
           return false;
         }
-        KeyValue firstKeyOfPreviousRow = KeyValue.createFirstOnRow(hfs
-            .getKeyValue().getRow());
+        KeyValue firstKeyOfPreviousRow = KeyValueUtil.createFirstOnRow(hfs.getKeyValue()
+            .getRowArray(), hfs.getKeyValue().getRowOffset(), hfs.getKeyValue().getRowLength());
 
         if (seekCount != null) seekCount.incrementAndGet();
         if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
@@ -430,10 +434,7 @@ public class StoreFileScanner implements
           this.stopSkippingKVsIfNextRow = false;
         }
         if (!resultOfSkipKVs
-            || Bytes.compareTo(cur.getBuffer(), cur.getRowOffset(),
-                cur.getRowLength(), firstKeyOfPreviousRow.getBuffer(),
-                firstKeyOfPreviousRow.getRowOffset(),
-                firstKeyOfPreviousRow.getRowLength()) > 0) {
+            || KeyValue.COMPARATOR.compareRows(cur, firstKeyOfPreviousRow) > 0) {
           return seekToPreviousRow(firstKeyOfPreviousRow);
         }
 
@@ -453,7 +454,7 @@ public class StoreFileScanner implements
     if (lastRow == null) {
       return false;
     }
-    KeyValue seekKey = KeyValue.createFirstOnRow(lastRow);
+    KeyValue seekKey = KeyValueUtil.createFirstOnRow(lastRow);
     if (seek(seekKey)) {
       return true;
     } else {
@@ -462,7 +463,7 @@ public class StoreFileScanner implements
   }
 
   @Override
-  public boolean backwardSeek(KeyValue key) throws IOException {
+  public boolean backwardSeek(Cell key) throws IOException {
     seek(key);
     if (cur == null
         || Bytes.compareTo(cur.getRowArray(), cur.getRowOffset(),

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java Wed Apr  9 11:46:14 2014
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.regionse
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/ParallelSeekHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/ParallelSeekHandler.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/ParallelSeekHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/ParallelSeekHandler.java Wed Apr  9 11:46:14 2014
@@ -24,11 +24,10 @@ import java.util.concurrent.CountDownLat
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
 
 /**
  * Handler to seek storefiles in parallel.
@@ -37,12 +36,12 @@ import org.apache.hadoop.hbase.regionser
 public class ParallelSeekHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(ParallelSeekHandler.class);
   private KeyValueScanner scanner;
-  private KeyValue keyValue;
+  private Cell keyValue;
   private long readPoint;
   private CountDownLatch latch;
   private Throwable err = null;
 
-  public ParallelSeekHandler(KeyValueScanner scanner,KeyValue keyValue,
+  public ParallelSeekHandler(KeyValueScanner scanner,Cell keyValue, 
       long readPoint, CountDownLatch latch) {
     super(null, EventType.RS_PARALLEL_SEEK);
     this.scanner = scanner;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java Wed Apr  9 11:46:14 2014
@@ -1437,7 +1437,7 @@ public class HLogSplitter {
           // We don't handle HBASE-2231 because we may or may not replay a compaction event.
           // Details at https://issues.apache.org/jira/browse/HBASE-2231?focusedCommentId=13647143&
           // page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13647143
-          if (kv.matchingFamily(WALEdit.METAFAMILY)) {
+          if (CellUtil.matchingFamily(kv, WALEdit.METAFAMILY)) {
             skippedKVs.add(kv);
             continue;
           }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java Wed Apr  9 11:46:14 2014
@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -265,7 +266,7 @@ public class WALEdit implements Writable
    * @return deserialized CompactionDescriptor or null.
    */
   public static CompactionDescriptor getCompaction(KeyValue kv) throws IOException {
-    if (kv.matchingRow(METAROW) && kv.matchingColumn(METAFAMILY, COMPACTION)) {
+    if (CellUtil.matchingRow(kv, METAROW) && CellUtil.matchingColumn(kv, METAFAMILY, COMPACTION)) {
       return CompactionDescriptor.parseFrom(kv.getValue());
     }
     return null;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java Wed Apr  9 11:46:14 2014
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -230,7 +231,7 @@ public class WALEditsReplaySink {
         List<KeyValue> kvs = edit.getKeyValues();
         for (KeyValue kv : kvs) {
           // filtering HLog meta entries
-          if (kv.matchingFamily(WALEdit.METAFAMILY)) continue;
+          if (CellUtil.matchingFamily(kv, WALEdit.METAFAMILY)) continue;
 
           setLocation(conn.locateRegion(tableName, kv.getRow()));
           skip = true;

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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java Wed Apr  9 11:46:14 2014
@@ -40,6 +40,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
@@ -672,7 +673,7 @@ public class ReplicationSource extends T
     int distinctRowKeys = 1;
     KeyValue lastKV = kvs.get(0);
     for (int i = 0; i < edit.size(); i++) {
-      if (!kvs.get(i).matchingRow(lastKV)) {
+      if (!CellUtil.matchingRow(kvs.get(i), lastKV)) {
         distinctRowKeys++;
       }
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java Wed Apr  9 11:46:14 2014
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.SortedSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
 
@@ -99,14 +100,14 @@ public class CollectionBackedScanner ext
   }
 
   @Override
-  public boolean seek(KeyValue seekKv) {
+  public boolean seek(Cell seekKv) {
     // restart iterator
     iter = data.iterator();
     return reseek(seekKv);
   }
 
   @Override
-  public boolean reseek(KeyValue seekKv) {
+  public boolean reseek(Cell seekKv) {
     while(iter.hasNext()){
       KeyValue next = iter.next();
       int ret = comparator.compare(next, seekKv);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java Wed Apr  9 11:46:14 2014
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.util;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
-import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.hbase.KeyValueUtil;
 
 @InterfaceAudience.Private
 public class CompoundBloomFilterBase implements BloomFilterBase {
@@ -84,7 +84,7 @@ public class CompoundBloomFilterBase imp
 
     // Make sure this does not specify a timestamp so that the default maximum
     // (most recent) timestamp is used.
-    KeyValue kv = KeyValue.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0,
+    KeyValue kv = KeyValueUtil.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0,
         qualifier, qoffset, qlength);
     return kv.getKey();
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java Wed Apr  9 11:46:14 2014
@@ -30,8 +30,10 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -121,7 +123,7 @@ public class TestHalfStoreFileReader {
     final HFileScanner scanner = halfreader.getScanner(false, false);
 
     scanner.seekTo();
-    KeyValue curr;
+    Cell curr;
     do {
       curr = scanner.getKeyValue();
       KeyValue reseekKv =
@@ -183,7 +185,7 @@ public class TestHalfStoreFileReader {
 
 
       // Seek on the splitKey, should be in top, not in bottom
-      KeyValue foundKeyValue = doTestOfSeekBefore(p, fs, bottom, midKV, cacheConf);
+      Cell foundKeyValue = doTestOfSeekBefore(p, fs, bottom, midKV, cacheConf);
       assertEquals(beforeMidKey, foundKeyValue);
 
       // Seek tot the last thing should be the penultimate on the top, the one before the midkey on the bottom.
@@ -213,7 +215,7 @@ public class TestHalfStoreFileReader {
       assertNull(foundKeyValue);
     }
 
-  private KeyValue doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, KeyValue seekBefore,
+  private Cell doTestOfSeekBefore(Path p, FileSystem fs, Reference bottom, KeyValue seekBefore,
                                         CacheConfig cacheConfig)
             throws IOException {
       final HalfStoreFileReader halfreader = new HalfStoreFileReader(fs, p,
@@ -224,8 +226,8 @@ public class TestHalfStoreFileReader {
       return scanner.getKeyValue();
   }
 
-  private KeyValue getLastOnCol(KeyValue curr) {
-    return KeyValue.createLastOnRow(
+  private KeyValue getLastOnCol(Cell curr) {
+    return KeyValueUtil.createLastOnRow(
         curr.getRowArray(), curr.getRowOffset(), curr.getRowLength(),
         curr.getFamilyArray(), curr.getFamilyOffset(), curr.getFamilyLength(),
         curr.getQualifierArray(), curr.getQualifierOffset(), curr.getQualifierLength());

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java Wed Apr  9 11:46:14 2014
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -252,7 +253,7 @@ public class TestDataBlockEncoders {
     for (boolean seekBefore : new boolean[] { false, true }) {
       checkSeekingConsistency(encodedSeekers, seekBefore, sampleKv.get(sampleKv.size() - 1));
       KeyValue midKv = sampleKv.get(sampleKv.size() / 2);
-      KeyValue lastMidKv = midKv.createLastOnRowCol();
+      KeyValue lastMidKv =KeyValueUtil.createLastOnRowCol(midKv);
       checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv);
     }
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java Wed Apr  9 11:46:14 2014
@@ -35,7 +35,9 @@ import java.util.concurrent.ConcurrentSk
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec;
@@ -113,14 +115,14 @@ public class TestPrefixTreeEncoding {
     seeker.setCurrentBuffer(readBuffer);
 
     // Seek before the first keyvalue;
-    KeyValue seekKey = KeyValue.createFirstDeleteFamilyOnRow(getRowKey(batchId, 0), CF_BYTES);
+    KeyValue seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(getRowKey(batchId, 0), CF_BYTES);
     seeker.seekToKeyInBlock(
         new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
             .getKeyLength()), true);
     assertEquals(null, seeker.getKeyValue());
 
     // Seek before the middle keyvalue;
-    seekKey = KeyValue.createFirstDeleteFamilyOnRow(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3),
+    seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3),
         CF_BYTES);
     seeker.seekToKeyInBlock(
         new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
@@ -129,7 +131,7 @@ public class TestPrefixTreeEncoding {
     assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3 - 1), seeker.getKeyValue().getRow());
 
     // Seek before the last keyvalue;
-    seekKey = KeyValue.createFirstDeleteFamilyOnRow(Bytes.toBytes("zzzz"), CF_BYTES);
+    seekKey = KeyValueUtil.createFirstDeleteFamilyOnRow(Bytes.toBytes("zzzz"), CF_BYTES);
     seeker.seekToKeyInBlock(
         new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
             .getKeyLength()), true);
@@ -156,9 +158,9 @@ public class TestPrefixTreeEncoding {
     ByteBuffer readBuffer = ByteBuffer.wrap(onDiskBytes, DataBlockEncoding.ID_SIZE,
         onDiskBytes.length - DataBlockEncoding.ID_SIZE);
     seeker.setCurrentBuffer(readBuffer);
-    KeyValue previousKV = null;
+    Cell previousKV = null;
     do {
-      KeyValue currentKV = seeker.getKeyValue();
+      Cell currentKV = seeker.getKeyValue();
       System.out.println(currentKV);
       if (previousKV != null && KeyValue.COMPARATOR.compare(currentKV, previousKV) < 0) {
         dumpInputKVSet();
@@ -223,7 +225,7 @@ public class TestPrefixTreeEncoding {
     for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
       kvList.clear();
       encodeSeeker.setCurrentBuffer(encodedData);
-      KeyValue firstOnRow = KeyValue.createFirstOnRow(getRowKey(batchId, i));
+      KeyValue firstOnRow = KeyValueUtil.createFirstOnRow(getRowKey(batchId, i));
       encodeSeeker.seekToKeyInBlock(
           new KeyValue.KeyOnlyKeyValue(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
               firstOnRow.getKeyLength()), false);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekToBlockWithEncoders.java Wed Apr  9 11:46:14 2014
@@ -23,8 +23,10 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -154,7 +156,7 @@ public class TestSeekToBlockWithEncoders
         Bytes.toBytes("q1"), Bytes.toBytes("val"));
     sampleKv.add(kv4);
     ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
-    KeyValue toSeek = KeyValue.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
+    KeyValue toSeek = KeyValueUtil.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
         kv3.getRowLength(), null, 0, 0, null, 0, 0);
     seekToTheKey(kv3, originalBuffer, toSeek);
   }
@@ -304,7 +306,7 @@ public class TestSeekToBlockWithEncoders
       seeker.seekToKeyInBlock(
           new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue
               .getKeyLength()), false);
-      KeyValue keyValue2 = seeker.getKeyValue();
+      Cell keyValue2 = seeker.getKeyValue();
       assertEquals(expected, keyValue2);
       seeker.rewind();
     }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java Wed Apr  9 11:46:14 2014
@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -540,7 +541,7 @@ public class TestHFileBlockIndex {
           byte[] row = TestHFileWriterV2.randomOrderedKey(rand, i);
 
           // Key will be interpreted by KeyValue.KEY_COMPARATOR
-          byte[] k = KeyValue.createFirstOnRow(row, 0, row.length, row, 0, 0,
+          byte[] k = KeyValueUtil.createFirstOnRow(row, 0, row.length, row, 0, 0,
               row, 0, 0).getKey();
 
           byte[] v = TestHFileWriterV2.randomValue(rand);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java?rev=1585945&r1=1585944&r2=1585945&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java Wed Apr  9 11:46:14 2014
@@ -36,9 +36,11 @@ import org.apache.hadoop.fs.FSDataInputS
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Cipher;
@@ -220,8 +222,9 @@ public class TestHFileEncryption {
         assertTrue("Initial seekTo failed", scanner.seekTo());
         int i = 0;
         do {
-          KeyValue kv = scanner.getKeyValue();
-          assertTrue("Read back an unexpected or invalid KV", testKvs.contains(kv));
+          Cell kv = scanner.getKeyValue();
+          assertTrue("Read back an unexpected or invalid KV",
+              testKvs.contains(KeyValueUtil.ensureKeyValue(kv)));
           i++;
         } while (scanner.next());
         reader.close();