Index: src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java (revision 1190164) +++ src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java (working copy) @@ -52,7 +52,7 @@ curVals.clear(); done = scanner.next(curVals); KeyValue kv = curVals.get(0); - sumResult += Bytes.toInt(kv.getValue()); + sumResult += Bytes.toInt(kv.getBuffer(), kv.getValueOffset()); } while (done); } finally { scanner.close(); Index: src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java (revision 1190164) +++ src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java (working copy) @@ -39,9 +39,9 @@ public Long getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv) throws IOException { - if (kv == null || kv.getValue().length != Bytes.SIZEOF_LONG) + if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG) return null; - return Bytes.toLong(kv.getValue()); + return Bytes.toLong(kv.getBuffer(), kv.getValueOffset()); } @Override Index: src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 1190164) +++ src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -1949,7 +1949,9 @@ if (result.size() == 0 && valueIsNull) { matches = true; } else if (result.size() == 1 && !valueIsNull) { - int compareResult = comparator.compareTo(result.get(0).getValue()); + KeyValue kv = result.get(0); + int compareResult = comparator.compareTo(kv.getBuffer(), + kv.getValueOffset(), kv.getValueLength()); switch (compareOp) { case LESS: matches = compareResult <= 0; @@ -3770,7 +3772,8 @@ long amount = column.getValue(); if (idx < results.size() && results.get(idx).matchingQualifier(column.getKey())) { - amount += Bytes.toLong(results.get(idx).getValue()); + KeyValue kv = results.get(idx); + amount += Bytes.toLong(kv.getBuffer(), kv.getValueOffset()); idx++; } Index: src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (revision 1190164) +++ src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (working copy) @@ -139,9 +139,11 @@ // include non-matches for the time being, they'll be discarded afterwards return ReturnCode.INCLUDE; } - // If it doesn't pass the op, skip it - if(comparator != null && doCompare(compareOp, comparator, v.getValue(), 0, v.getValueLength())) - return ReturnCode.SKIP; + // If it doesn't pass the op, skip it + if (comparator != null + && doCompare(compareOp, comparator, v.getBuffer(), v.getValueOffset(), + v.getValueLength())) + return ReturnCode.SKIP; stampSet.add(v.getTimestamp()); if(dropDependentColumn) { Index: src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java (revision 1190164) +++ src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java (working copy) @@ -126,17 +126,17 @@ } // With mini-batching, we need to expect multiple rows per edit byte[] lastKey = kvs.get(0).getRow(); - Put put = new Put(kvs.get(0).getRow(), - kvs.get(0).getTimestamp()); + Put put = new Put(lastKey, kvs.get(0).getTimestamp()); put.setClusterId(entry.getKey().getClusterId()); for (KeyValue kv : kvs) { - if (!Bytes.equals(lastKey, kv.getRow())) { + byte[] key = kv.getRow(); + if (!Bytes.equals(lastKey, key)) { tableList.add(put); - put = new Put(kv.getRow(), kv.getTimestamp()); + put = new Put(key, kv.getTimestamp()); put.setClusterId(entry.getKey().getClusterId()); } - put.add(kv.getFamily(), kv.getQualifier(), kv.getValue()); - lastKey = kv.getRow(); + put.add(kv); + lastKey = key; } tableList.add(put); }