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

svn commit: r1519076 [2/6] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/ hbase-client/src...

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

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java Fri Aug 30 20:31:44 2013
@@ -18,15 +18,16 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that will only return the first KV from each row.
@@ -45,7 +46,8 @@ public class FirstKeyOnlyFilter extends 
     foundKV = false;
   }
 
-  public ReturnCode filterKeyValue(KeyValue v) {
+  @Override
+  public ReturnCode filterKeyValue(Cell v) {
     if(foundKV) return ReturnCode.NEXT_ROW;
     foundKV = true;
     return ReturnCode.INCLUDE;

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java Fri Aug 30 20:31:44 2013
@@ -22,7 +22,9 @@ import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -57,7 +59,8 @@ public class FirstKeyValueMatchingQualif
     this.qualifiers = qualifiers;
   }
 
-  public ReturnCode filterKeyValue(KeyValue v) {
+  @Override
+  public ReturnCode filterKeyValue(Cell v) {
     if (hasFoundKV()) {
       return ReturnCode.NEXT_ROW;
     } else if (hasOneMatchingQualifier(v)) {
@@ -66,9 +69,11 @@ public class FirstKeyValueMatchingQualif
     return ReturnCode.INCLUDE;
   }
 
-  private boolean hasOneMatchingQualifier(KeyValue v) {
+  private boolean hasOneMatchingQualifier(Cell v) {
     for (byte[] q : qualifiers) {
-      if (v.matchingQualifier(q)) {
+      // TODO get rid of this by adding matching qualifier to interface.
+      KeyValue kv = KeyValueUtil.ensureKeyValue(v);
+      if (kv.matchingQualifier(q)) {
         return true;
       }
     }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java Fri Aug 30 20:31:44 2013
@@ -21,7 +21,9 @@ import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
@@ -72,8 +74,11 @@ public class FuzzyRowFilter extends Filt
 
   // TODO: possible improvement: save which fuzzy row key to use when providing a hint
   @Override
-  public ReturnCode filterKeyValue(KeyValue kv) {
-    byte[] rowKey = kv.getRow();
+  public ReturnCode filterKeyValue(Cell kv) {
+    // TODO add getRow() equivalent to Cell or change satisfies to take b[],o,l style args.
+    KeyValue v = KeyValueUtil.ensureKeyValue(kv);
+
+    byte[] rowKey = v.getRow();
     // assigning "worst" result first and looking for better options
     SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
     for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
@@ -98,8 +103,11 @@ public class FuzzyRowFilter extends Filt
   }
 
   @Override
-  public KeyValue getNextKeyHint(KeyValue currentKV) {
-    byte[] rowKey = currentKV.getRow();
+  public Cell getNextCellHint(Cell currentKV) {
+    // TODO make matching Column a cell method or CellUtil method.
+    KeyValue v = KeyValueUtil.ensureKeyValue(currentKV);
+
+    byte[] rowKey = v.getRow();
     byte[] nextRowKey = null;
     // Searching for the "smallest" row key that satisfies at least one fuzzy row key
     for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Fri Aug 30 20:31:44 2013
@@ -19,15 +19,18 @@
 package org.apache.hadoop.hbase.filter;
 
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that will only return the key component of each KV (the value will
@@ -45,8 +48,13 @@ public class KeyOnlyFilter extends Filte
   public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
 
   @Override
-  public KeyValue transform(KeyValue kv) {
-    return kv.createKeyOnly(this.lenAsVal);
+  public Cell transformCell(Cell kv) {
+    // TODO Move to KeyValueUtil
+
+    // TODO make matching Column a cell method or CellUtil method.
+    KeyValue v = KeyValueUtil.ensureKeyValue(kv);
+
+    return v.createKeyOnly(this.lenAsVal);
   }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java Fri Aug 30 20:31:44 2013
@@ -21,6 +21,7 @@ import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -62,11 +63,11 @@ public class MultipleColumnPrefixFilter 
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue kv) {
-    if (sortedPrefixes.size() == 0 || kv.getBuffer() == null) {
+  public ReturnCode filterKeyValue(Cell kv) {
+    if (sortedPrefixes.size() == 0 || kv.getQualifierArray() == null) {
       return ReturnCode.INCLUDE;
     } else {
-      return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+      return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
     }
   }
 
@@ -152,9 +153,10 @@ public class MultipleColumnPrefixFilter 
     return this.sortedPrefixes.equals(other.sortedPrefixes);
   }
 
-  public KeyValue getNextKeyHint(KeyValue kv) {
+  @Override
+  public Cell getNextCellHint(Cell kv) {
     return KeyValue.createFirstOnRow(
-      kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
+      kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
       kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
   }
 

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java Fri Aug 30 20:31:44 2013
@@ -19,13 +19,13 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+
 /**
  * A binary comparator which lexicographically compares against the specified
  * byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java Fri Aug 30 20:31:44 2013
@@ -44,7 +44,7 @@ import java.util.Stack;
  * a filter object is constructed. This filter object is then wrapped
  * in a scanner object which is then returned
  * <p>
- * This class addresses the HBASE-4168 JIRA. More documentaton on this
+ * This class addresses the HBASE-4168 JIRA. More documentation on this
  * Filter Language can be found at: https://issues.apache.org/jira/browse/HBASE-4176
  */
 @InterfaceAudience.Public
@@ -242,8 +242,8 @@ public class ParseFilter {
     }
     try {
       filterName = filterHashMap.get(filterName);
-      Class c = Class.forName(filterName);
-      Class[] argTypes = new Class [] {ArrayList.class};
+      Class<?> c = Class.forName(filterName);
+      Class<?>[] argTypes = new Class [] {ArrayList.class};
       Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
       return (Filter) m.invoke(null,filterArguments);
     } catch (ClassNotFoundException e) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java Fri Aug 30 20:31:44 2013
@@ -19,17 +19,18 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.io.IOException;
-import java.util.ArrayList;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used to filter based on the column qualifier. It takes an
@@ -59,10 +60,10 @@ public class QualifierFilter extends Com
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell v) {
     int qualifierLength = v.getQualifierLength();
     if (qualifierLength > 0) {
-      if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+      if (doCompare(this.compareOp, this.comparator, v.getQualifierArray(),
           v.getQualifierOffset(), qualifierLength)) {
         return ReturnCode.SKIP;
       }
@@ -71,7 +72,7 @@ public class QualifierFilter extends Com
   }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
-    ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+    ArrayList<?> arguments = CompareFilter.extractArguments(filterArguments);
     CompareOp compareOp = (CompareOp)arguments.get(0);
     ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
     return new QualifierFilter(compareOp, comparator);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java Fri Aug 30 20:31:44 2013
@@ -19,14 +19,15 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.Random;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.util.Random;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that includes rows based on a chance.
@@ -71,7 +72,7 @@ public class RandomRowFilter extends Fil
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell v) {
     if (filterOutRow) {
       return ReturnCode.NEXT_ROW;
     }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java Fri Aug 30 20:31:44 2013
@@ -19,17 +19,18 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.io.IOException;
-import java.util.ArrayList;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used to filter based on the key. It takes an operator
@@ -65,7 +66,7 @@ public class RowFilter extends CompareFi
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell v) {
     if(this.filterOutRow) {
       return ReturnCode.NEXT_ROW;
     }
@@ -86,6 +87,7 @@ public class RowFilter extends CompareFi
   }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+    @SuppressWarnings("rawtypes") // for arguments
     ArrayList arguments = CompareFilter.extractArguments(filterArguments);
     CompareOp compareOp = (CompareOp)arguments.get(0);
     ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java Fri Aug 30 20:31:44 2013
@@ -22,7 +22,9 @@ package org.apache.hadoop.hbase.filter;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -100,10 +102,11 @@ public class SingleColumnValueExcludeFil
   }
 
   // Here we remove from row all key values from testing column
-  public void filterRow(List<KeyValue> kvs) {
-    Iterator it = kvs.iterator();
+  @Override
+  public void filterRowCells(List<Cell> kvs) {
+    Iterator<? extends Cell> it = kvs.iterator();
     while (it.hasNext()) {
-      KeyValue kv = (KeyValue)it.next();
+      KeyValue kv = KeyValueUtil.ensureKeyValue(it.next());
       // If the current column is actually the tested column,
       // we will skip it instead.
       if (kv.matchingColumn(this.columnFamily, this.columnQualifier)) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java Fri Aug 30 20:31:44 2013
@@ -26,7 +26,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -166,7 +168,11 @@ public class SingleColumnValueFilter ext
     return columnQualifier;
   }
 
-  public ReturnCode filterKeyValue(KeyValue keyValue) {
+  @Override
+  public ReturnCode filterKeyValue(Cell c) {
+    // TODO get rid of this.
+    KeyValue keyValue = KeyValueUtil.ensureKeyValue(c);
+    
     // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue()));
     if (this.matchedColumn) {
       // We already found and matched the single column, all keys now pass

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java Fri Aug 30 20:31:44 2013
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.protobuf.
 import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
- * A wrapper filter that filters an entire row if any of the KeyValue checks do
+ * A wrapper filter that filters an entire row if any of the Cell checks do
  * not pass.
  * <p>
  * For example, if all columns in a row represent weights of different things,
@@ -45,7 +45,7 @@ import com.google.protobuf.InvalidProtoc
  *     new BinaryComparator(Bytes.toBytes(0))));
  * </code>
  * Any row which contained a column whose value was 0 will be filtered out
- * (since ValueFilter will not pass that KeyValue).
+ * (since ValueFilter will not pass that Cell).
  * Without this filter, the other non-zero valued columns in the row would still
  * be emitted.
  */
@@ -74,15 +74,15 @@ public class SkipFilter extends FilterBa
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) throws IOException {
+  public ReturnCode filterKeyValue(Cell v) throws IOException {
     ReturnCode c = filter.filterKeyValue(v);
     changeFR(c != ReturnCode.INCLUDE);
     return c;
   }
 
   @Override
-  public KeyValue transform(KeyValue v) throws IOException {
-    return filter.transform(v);
+  public Cell transformCell(Cell v) throws IOException {
+    return filter.transformCell(v);
   }
 
   public boolean filterRow() {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java Fri Aug 30 20:31:44 2013
@@ -21,7 +21,7 @@ import com.google.common.base.Preconditi
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
@@ -88,7 +88,7 @@ public class TimestampsFilter extends Fi
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell v) {
     if (this.timestamps.contains(v.getTimestamp())) {
       return ReturnCode.INCLUDE;
     } else if (v.getTimestamp() < minTimeStamp) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java Fri Aug 30 20:31:44 2013
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -58,8 +58,8 @@ public class ValueFilter extends Compare
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
-    if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+  public ReturnCode filterKeyValue(Cell v) {
+    if (doCompare(this.compareOp, this.comparator, v.getValueArray(),
         v.getValueOffset(), v.getValueLength())) {
       return ReturnCode.SKIP;
     }
@@ -67,6 +67,7 @@ public class ValueFilter extends Compare
   }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+    @SuppressWarnings("rawtypes")  // for arguments
     ArrayList arguments = CompareFilter.extractArguments(filterArguments);
     CompareOp compareOp = (CompareOp)arguments.get(0);
     ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java Fri Aug 30 20:31:44 2013
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -33,7 +33,7 @@ import com.google.protobuf.InvalidProtoc
 /**
  * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon
  * as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)},
- * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)},
+ * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)},
  * {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or
  * {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods
  * returns true.
@@ -73,15 +73,15 @@ public class WhileMatchFilter extends Fi
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) throws IOException {
+  public ReturnCode filterKeyValue(Cell v) throws IOException {
     ReturnCode c = filter.filterKeyValue(v);
     changeFAR(c != ReturnCode.INCLUDE);
     return c;
   }
 
   @Override
-  public KeyValue transform(KeyValue v) throws IOException {
-    return filter.transform(v);
+  public Cell transformCell(Cell v) throws IOException {
+    return filter.transformCell(v);
   }
 
   @Override

Modified: hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java (original)
+++ hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java Fri Aug 30 20:31:44 2013
@@ -73,73 +73,73 @@ public class TestPayloadCarryingRpcContr
 
               @Override
               public byte[] getRowArray() {
-                // TODO Auto-generated method stub
+                // unused
                 return null;
               }
 
               @Override
               public int getRowOffset() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public short getRowLength() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public byte[] getFamilyArray() {
-                // TODO Auto-generated method stub
+                // unused
                 return null;
               }
 
               @Override
               public int getFamilyOffset() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public byte getFamilyLength() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public byte[] getQualifierArray() {
-                // TODO Auto-generated method stub
+                // unused
                 return null;
               }
 
               @Override
               public int getQualifierOffset() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public int getQualifierLength() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public long getTimestamp() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public byte getTypeByte() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public long getMvccVersion() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
@@ -160,19 +160,43 @@ public class TestPayloadCarryingRpcContr
 
               @Override
               public int getTagsOffset() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public short getTagsLength() {
-                // TODO Auto-generated method stub
+                // unused
                 return 0;
               }
 
               @Override
               public byte[] getTagsArray() {
-                // TODO Auto-generated method stub
+                // unused
+                return null;
+              }
+
+              @Override
+              public byte[] getValue() {
+                // unused
+                return null;
+              }
+
+              @Override
+              public byte[] getFamily() {
+                // unused
+                return null;
+              }
+
+              @Override
+              public byte[] getQualifier() {
+                // unused
+                return null;
+              }
+
+              @Override
+              public byte[] getRow() {
+                // unused
                 return null;
               }
             };

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

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java Fri Aug 30 20:31:44 2013
@@ -27,6 +27,7 @@ import java.util.NavigableMap;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -257,6 +258,41 @@ public final class CellUtil {
       right.getRowArray(), right.getRowOffset(), right.getRowLength());
   }
 
+  public static boolean matchingRow(final Cell left, final byte[] buf) {
+    return Bytes.equals(left.getRowArray(),  left.getRowOffset(), left.getRowLength(),
+      buf, 0, buf.length);
+  }
+
+  public static boolean matchingFamily(final Cell left, final Cell right) {
+    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+        right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
+  }
+
+  public static boolean matchingFamily(final Cell left, final byte[] buf) {
+    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+        buf, 0, buf.length);
+  }
+
+  public static boolean matchingQualifier(final Cell left, final Cell right) {
+    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
+        right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
+  }
+
+  public static boolean matchingQualifier(final Cell left, final byte[] buf) {
+    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
+        buf, 0, buf.length);
+  }
+
+
+  public static boolean matchingValue(final Cell left, final Cell right) {
+    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
+        right.getValueArray(), right.getValueOffset(), right.getValueLength());
+  }
+
+  public static boolean matchingValue(final Cell left, final byte[] buf) {
+    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
+        buf, 0, buf.length);
+  }
   /**
    * @return True if a delete type, a {@link KeyValue.Type#Delete} or
    * a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
@@ -266,6 +302,10 @@ public final class CellUtil {
     return KeyValue.isDelete(cell.getTypeByte());
   }
 
+  public static boolean isDeleteFamily(final Cell cell) {
+    return cell.getTypeByte() == Type.DeleteFamily.getCode();
+  }
+
   /**
    * @param cell
    * @return Estimate of the <code>cell</code> size in bytes.
@@ -286,4 +326,4 @@ public final class CellUtil {
       // Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
       Bytes.SIZEOF_INT;
   }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Fri Aug 30 20:31:44 2013
@@ -26,7 +26,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -36,7 +35,6 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -46,7 +44,10 @@ import org.apache.hadoop.io.RawComparato
 import com.google.common.primitives.Longs;
 
 /**
- * An HBase Key/Value. This is the fundamental HBase Type.
+ * An HBase Key/Value. This is the fundamental HBase Type.  
+ * <p>
+ * HBase applications and users should use the Cell interface and avoid directly using KeyValue
+ * and member functions not defined in Cell.
  * <p>
  * If being used client-side, the primary methods to access individual fields are {@link #getRow()},
  * {@link #getFamily()}, {@link #getQualifier()}, {@link #getTimestamp()}, and {@link #getValue()}.
@@ -66,8 +67,7 @@ import com.google.common.primitives.Long
  * <code>Integer.MAX_SIZE</code>. The column does not contain the family/qualifier delimiter,
  * {@link #COLUMN_FAMILY_DELIMITER}
  */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceAudience.Private
 public class KeyValue implements Cell, HeapSize, Cloneable {
   static final Log LOG = LogFactory.getLog(KeyValue.class);
 
@@ -472,6 +472,14 @@ public class KeyValue implements Cell, H
     this.offset = 0;
   }
 
+  public KeyValue(Cell c) {
+    this(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
+        c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(), 
+        c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
+        c.getTimestamp(), Type.codeToType(c.getTypeByte()), 
+        c.getValueArray(), c.getValueOffset(), c.getValueLength());
+  }
+  
   /**
    * Create an empty byte[] representing a KeyValue
    * All lengths are preset and can be filled in later.
@@ -1175,34 +1183,9 @@ public class KeyValue implements Cell, H
    * save on allocations.
    * @return Value in a new byte array.
    */
+  @Deprecated // use CellUtil.getValueArray()
   public byte [] getValue() {
-    int o = getValueOffset();
-    int l = getValueLength();
-    byte [] result = new byte[l];
-    System.arraycopy(getBuffer(), o, result, 0, l);
-    return result;
-  }
-
-  /**
-   * Returns the value wrapped in a new <code>ByteBuffer</code>.
-   *
-   * @return the value
-   */
-  public ByteBuffer getValueAsByteBuffer() {
-    return ByteBuffer.wrap(getBuffer(), getValueOffset(), getValueLength());
-  }
-
-  /**
-   * Loads this object's value into the provided <code>ByteBuffer</code>.
-   * <p>
-   * Does not clear or flip the buffer.
-   *
-   * @param dst the buffer where to write the value
-   *
-   * @throws BufferOverflowException if there is insufficient space remaining in the buffer
-   */
-  public void loadValue(ByteBuffer dst) throws BufferOverflowException {
-    dst.put(getBuffer(), getValueOffset(), getValueLength());
+    return CellUtil.getValueArray(this);
   }
 
   /**
@@ -1213,12 +1196,9 @@ public class KeyValue implements Cell, H
    * lengths instead.
    * @return Row in a new byte array.
    */
+  @Deprecated // use CellUtil.getRowArray()
   public byte [] getRow() {
-    int o = getRowOffset();
-    short l = getRowLength();
-    byte result[] = new byte[l];
-    System.arraycopy(getBuffer(), o, result, 0, l);
-    return result;
+    return CellUtil.getRowArray(this);
   }
 
   /**
@@ -1260,6 +1240,7 @@ public class KeyValue implements Cell, H
    * a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
    * KeyValue type.
    */
+  @Deprecated // use CellUtil#isDelete
   public boolean isDelete() {
     return KeyValue.isDelete(getType());
   }
@@ -1303,12 +1284,9 @@ public class KeyValue implements Cell, H
    * lengths instead.
    * @return Returns family. Makes a copy.
    */
+  @Deprecated // use CellUtil.getFamilyArray
   public byte [] getFamily() {
-    int o = getFamilyOffset();
-    int l = getFamilyLength(o);
-    byte [] result = new byte[l];
-    System.arraycopy(this.bytes, o, result, 0, l);
-    return result;
+    return CellUtil.getFamilyArray(this);
   }
 
   /**
@@ -1320,12 +1298,9 @@ public class KeyValue implements Cell, H
    * Use {@link #getBuffer()} with appropriate offsets and lengths instead.
    * @return Returns qualifier. Makes a copy.
    */
+  @Deprecated // use CellUtil.getQualifierArray
   public byte [] getQualifier() {
-    int o = getQualifierOffset();
-    int l = getQualifierLength();
-    byte [] result = new byte[l];
-    System.arraycopy(this.bytes, o, result, 0, l);
-    return result;
+    return CellUtil.getQualifierArray(this);
   }
 
   //---------------------------------------------------------------------------
@@ -1419,6 +1394,8 @@ public class KeyValue implements Cell, H
   /**
    * Creates a new KeyValue that only contains the key portion (the value is
    * set to be null).
+   *
+   * TODO only used by KeyOnlyFilter -- move there.
    * @param lenAsVal replace value with the actual value length (false=empty)
    */
   public KeyValue createKeyOnly(boolean lenAsVal) {
@@ -2586,4 +2563,5 @@ public class KeyValue implements Cell, H
   public byte[] getTagsArray() {
     throw new UnsupportedOperationException("Not implememnted");
   }
+
 }

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java Fri Aug 30 20:31:44 2013
@@ -80,11 +80,11 @@ public class KeyValueTestUtil {
    * @param kvCollection2
    * @return true if KeyValues from kvCollection2 are contained in kvCollection1
    */
-  public static boolean containsIgnoreMvccVersion(Collection<KeyValue> kvCollection1,
-      Collection<KeyValue> kvCollection2) {
-    for (KeyValue kv1 : kvCollection1) {
+  public static boolean containsIgnoreMvccVersion(Collection<? extends Cell> kvCollection1,
+      Collection<? extends Cell> kvCollection2) {
+    for (Cell kv1 : kvCollection1) {
       boolean found = false;
-      for (KeyValue kv2 : kvCollection2) {
+      for (Cell kv2 : kvCollection2) {
         if (CellComparator.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
       }
       if (!found) return false;

Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java Fri Aug 30 20:31:44 2013
@@ -26,6 +26,7 @@ import java.util.Iterator;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -73,7 +74,7 @@ public class EncodedDataBlock {
    * @param headerSize header size of the block.
    * @return Forwards sequential iterator.
    */
-  public Iterator<KeyValue> getIterator(int headerSize) {
+  public Iterator<Cell> getIterator(int headerSize) {
     final int rawSize = rawKVs.length;
     byte[] encodedDataWithHeader = getEncodedData();
     int bytesToSkip = headerSize + Bytes.SIZEOF_SHORT;
@@ -81,7 +82,7 @@ public class EncodedDataBlock {
         bytesToSkip, encodedDataWithHeader.length - bytesToSkip);
     final DataInputStream dis = new DataInputStream(bais);
 
-    return new Iterator<KeyValue>() {
+    return new Iterator<Cell>() {
       private ByteBuffer decompressedData = null;
 
       @Override
@@ -93,7 +94,7 @@ public class EncodedDataBlock {
       }
 
       @Override
-      public KeyValue next() {
+      public Cell next() {
         if (decompressedData == null) {
           try {
             decompressedData = dataBlockEncoder.decodeKeyValues(

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

Modified: hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java (original)
+++ hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java Fri Aug 30 20:31:44 2013
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.hbase.coprocessor.example;
 
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@@ -34,9 +36,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
 
 /**
  * Sample coprocessor endpoint exposing a Service interface for counting rows and key values.
@@ -73,14 +75,14 @@ public class RowCountEndpoint extends Ex
     InternalScanner scanner = null;
     try {
       scanner = env.getRegion().getScanner(scan);
-      List<KeyValue> results = new ArrayList<KeyValue>();
+      List<Cell> results = new ArrayList<Cell>();
       boolean hasMore = false;
       byte[] lastRow = null;
       long count = 0;
       do {
         hasMore = scanner.next(results);
-        for (KeyValue kv : results) {
-          byte[] currentRow = kv.getRow();
+        for (Cell kv : results) {
+          byte[] currentRow = CellUtil.getRowArray(kv);
           if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
             lastRow = currentRow;
             count++;
@@ -113,12 +115,12 @@ public class RowCountEndpoint extends Ex
     InternalScanner scanner = null;
     try {
       scanner = env.getRegion().getScanner(new Scan());
-      List<KeyValue> results = new ArrayList<KeyValue>();
+      List<Cell> results = new ArrayList<Cell>();
       boolean hasMore = false;
       long count = 0;
       do {
         hasMore = scanner.next(results);
-        for (KeyValue kv : results) {
+        for (Cell kv : results) {
           count++;
         }
         results.clear();

Modified: hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java (original)
+++ hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java Fri Aug 30 20:31:44 2013
@@ -25,11 +25,12 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
@@ -39,22 +40,18 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
-import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
-import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
 import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder;
 import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
+import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
 @Category(MediumTests.class)
@@ -276,17 +273,17 @@ public class TestBulkDeleteProtocol {
     scan.setMaxVersions();
     for (Result result : ht.getScanner(scan)) {
       assertEquals(3, result.getFamilyMap(FAMILY1).size());
-      List<KeyValue> column = result.getColumn(FAMILY1, QUALIFIER1);
+      List<Cell> column = result.getColumn(FAMILY1, QUALIFIER1);
       assertEquals(1, column.size());
-      assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
+      assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
 
       column = result.getColumn(FAMILY1, QUALIFIER2);
       assertEquals(1, column.size());
-      assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
+      assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
 
       column = result.getColumn(FAMILY1, QUALIFIER3);
       assertEquals(1, column.size());
-      assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
+      assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
       rows++;
     }
     assertEquals(100, rows);
@@ -330,10 +327,10 @@ public class TestBulkDeleteProtocol {
       assertEquals(3, result.getFamilyMap(FAMILY1).size());
       assertEquals(3, result.getColumn(FAMILY1, QUALIFIER1).size());
       assertEquals(3, result.getColumn(FAMILY1, QUALIFIER2).size());
-      List<KeyValue> column = result.getColumn(FAMILY1, QUALIFIER3);
+      List<Cell> column = result.getColumn(FAMILY1, QUALIFIER3);
       assertEquals(2, column.size());
-      assertTrue(Bytes.equals("v3".getBytes(), column.get(0).getValue()));
-      assertTrue(Bytes.equals("v1".getBytes(), column.get(1).getValue()));
+      assertTrue(CellUtil.matchingValue(column.get(0), "v3".getBytes()));
+      assertTrue(CellUtil.matchingValue(column.get(1), "v1".getBytes()));
       rows++;
     }
     assertEquals(100, rows);
@@ -410,14 +407,14 @@ public class TestBulkDeleteProtocol {
     scan1.setMaxVersions();
     for (Result res : ht.getScanner(scan1)) {
       assertEquals(3, res.getFamilyMap(FAMILY1).size());
-      List<KeyValue> column = res.getColumn(FAMILY1, QUALIFIER1);
+      List<Cell> column = res.getColumn(FAMILY1, QUALIFIER1);
       assertEquals(2, column.size());
-      assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue()));
-      assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue()));
+      assertTrue(CellUtil.matchingValue(column.get(0), "v4".getBytes()));
+      assertTrue(CellUtil.matchingValue(column.get(1), "v3".getBytes()));
       column = res.getColumn(FAMILY1, QUALIFIER2);
       assertEquals(2, column.size());
-      assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue()));
-      assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue()));
+      assertTrue(CellUtil.matchingValue(column.get(0), "v4".getBytes()));
+      assertTrue(CellUtil.matchingValue(column.get(1), "v3".getBytes()));
       assertEquals(4, res.getColumn(FAMILY1, QUALIFIER3).size());
       rows++;
     }

Modified: hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java (original)
+++ hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java Fri Aug 30 20:31:44 2013
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -30,6 +32,8 @@ import java.util.Set;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.IntegrationTestBase;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
@@ -65,8 +69,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.assertEquals;
-
 /**
  * Test Bulk Load and MR on a distributed cluster.
  * It starts an MR job that creates linked chains
@@ -468,7 +470,8 @@ public class IntegrationTestBulkLoad ext
       for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
         long chainId = Bytes.toLong(entry.getKey());
         long next = Bytes.toLong(entry.getValue());
-        long order = Bytes.toLong(value.getColumn(SORT_FAM, entry.getKey()).get(0).getValue());
+        Cell c = value.getColumn(SORT_FAM, entry.getKey()).get(0);
+        long order = Bytes.toLong(CellUtil.getValueArray(c));
         context.write(new LinkKey(chainId, order), new LinkChain(longRk, next));
       }
     }

Modified: hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java (original)
+++ hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java Fri Aug 30 20:31:44 2013
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Iterator;
@@ -31,6 +36,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -62,11 +68,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static java.lang.String.format;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Validate ImportTsv + LoadIncrementalHFiles on a distributed cluster.
  */
@@ -153,7 +154,7 @@ public class IntegrationTestImportTsv im
       Iterator<KeyValue> expectedIt = simple_expected.iterator();
       while (resultsIt.hasNext() && expectedIt.hasNext()) {
         Result r = resultsIt.next();
-        for (KeyValue actual : r.raw()) {
+        for (Cell actual : r.raw()) {
           assertTrue(
             "Ran out of expected values prematurely!",
             expectedIt.hasNext());

Modified: hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java (original)
+++ hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java Fri Aug 30 20:31:44 2013
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hbase.test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.Random;
 import java.util.Set;
@@ -24,13 +27,13 @@ import java.util.UUID;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.google.common.collect.Sets;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -38,7 +41,6 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.IntegrationTestBase;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.IntegrationTests;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -65,8 +67,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import com.google.common.collect.Sets;
 
 /**
  * A large test which loads a lot of data that has internal references, and
@@ -247,12 +248,12 @@ public class IntegrationTestLoadAndVerif
         throws IOException, InterruptedException {
       BytesWritable bwKey = new BytesWritable(key.get());
       BytesWritable bwVal = new BytesWritable();
-      for (KeyValue kv : value.list()) {
+      for (Cell kv : value.list()) {
         if (Bytes.compareTo(TEST_QUALIFIER, 0, TEST_QUALIFIER.length,
-                            kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
+                            kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
           context.write(bwKey, EMPTY);
         } else {
-          bwVal.set(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+          bwVal.set(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
           context.write(bwVal, bwKey);
         }
       }

Modified: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java (original)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java Fri Aug 30 20:31:44 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.pr
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 
@@ -183,6 +184,26 @@ public class PrefixTreeCell implements C
     return type.getCode();
   }
 
+  /* Deprecated methods pushed into the Cell interface */
+  @Override
+  public byte[] getValue() {
+    return CellUtil.getValueArray(this);
+  }
+
+  @Override
+  public byte[] getFamily() {
+    return CellUtil.getFamilyArray(this);
+  }
+
+  @Override
+  public byte[] getQualifier() {
+    return CellUtil.getQualifierArray(this);
+  }
+
+  @Override
+  public byte[] getRow() {
+    return CellUtil.getRowArray(this);
+  }
 
   /************************* helper methods *************************/
 
@@ -208,4 +229,5 @@ public class PrefixTreeCell implements C
   public byte[] getTagsArray() {
     throw new UnsupportedOperationException("Not implemented");
   }
+
 }

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

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java Fri Aug 30 20:31:44 2013
@@ -21,6 +21,7 @@ import java.util.NavigableSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
@@ -429,6 +430,15 @@ public interface RegionObserver extends 
    * is not bypassed.
    * @throws IOException if an error occurred on the coprocessor
    */
+  void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
+      final List<Cell> result)
+    throws IOException;
+
+  /**
+   * WARNING: please override preGetOp instead of this method.  This is to maintain some
+   * compatibility and to ease the transition from 0.94 -> 0.96.
+   */
+  @Deprecated
   void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
       final List<KeyValue> result)
     throws IOException;
@@ -443,6 +453,15 @@ public interface RegionObserver extends 
    * @param result the result to return to the client, modify as necessary
    * @throws IOException if an error occurred on the coprocessor
    */
+  void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
+      final List<Cell> result)
+    throws IOException;
+
+  /**
+   * WARNING: please override postGetOp instead of this method.  This is to maintain some
+   * compatibility and to ease the transition from 0.94 -> 0.96.
+   */
+  @Deprecated
   void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
       final List<KeyValue> result)
     throws IOException;

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java Fri Aug 30 20:31:44 2013
@@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.mapred;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Map;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceBase;
@@ -115,12 +116,12 @@ implements TableMap<ImmutableBytesWritab
     ArrayList<byte[]> foundList = new ArrayList<byte[]>();
     int numCols = columns.length;
     if (numCols > 0) {
-      for (KeyValue value: r.list()) {
-        byte [] column = KeyValue.makeColumn(value.getFamily(),
-            value.getQualifier());
+      for (Cell value: r.list()) {
+        byte [] column = KeyValue.makeColumn(CellUtil.getFamilyArray(value),
+            CellUtil.getQualifierArray(value));
         for (int i = 0; i < numCols; i++) {
           if (Bytes.equals(column, columns[i])) {
-            foundList.add(value.getValue());
+            foundList.add(CellUtil.getValueArray(value));
             break;
           }
         }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java Fri Aug 30 20:31:44 2013
@@ -26,8 +26,9 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.CompareFilter;
@@ -115,9 +116,9 @@ public class CellCounter {
         context.getCounter(Counters.ROWS).increment(1);
         context.write(new Text("Total ROWS"), new IntWritable(1));
 
-        for (KeyValue value : values.list()) {
-          currentRowKey = Bytes.toStringBinary(value.getRow());
-          String thisRowFamilyName = Bytes.toStringBinary(value.getFamily());
+        for (Cell value : values.list()) {
+          currentRowKey = Bytes.toStringBinary(CellUtil.getRowArray(value));
+          String thisRowFamilyName = Bytes.toStringBinary(CellUtil.getFamilyArray(value));
           if (thisRowFamilyName != null &&
               !thisRowFamilyName.equals(currentFamilyName)) {
             currentFamilyName = thisRowFamilyName;
@@ -127,7 +128,7 @@ public class CellCounter {
             context.write(new Text(thisRowFamilyName), new IntWritable(1));
           }
           String thisRowQualifierName = thisRowFamilyName + separator
-              + Bytes.toStringBinary(value.getQualifier());
+              + Bytes.toStringBinary(CellUtil.getQualifierArray(value));
           if (thisRowQualifierName != null &&
               !thisRowQualifierName.equals(currentQualifierName)) {
             currentQualifierName = thisRowQualifierName;