hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jmhs...@apache.org
Subject svn commit: r1519076 [1/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...
Date Fri, 30 Aug 2013 20:31:48 GMT
Author: jmhsieh
Date: Fri Aug 30 20:31:44 2013
New Revision: 1519076

URL: http://svn.apache.org/r1519076
Log:
HBASE-9334 Convert KeyValue to Cell in hbase-client module - Filters  
HBASE-9359 Convert KeyValue to Cell in hbase-client module - Result/Put/Delete, ColumnInterpreter

Modified:
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
    hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
    hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
    hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
    hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java
    hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
    hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java
    hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
    hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
    hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
    hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/GroupingTableMapper.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ResultGenerator.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerInstanceResource.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ScannerResultGenerator.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/MetaMockingUtil.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java Fri Aug 30 20:31:44 2013
@@ -15,11 +15,11 @@
 
 package org.apache.hadoop.hbase;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-import java.io.IOException;
-
 /**
  * Coprocess interface.
  */
@@ -53,4 +53,5 @@ public interface Coprocessor {
   void start(CoprocessorEnvironment env) throws IOException;
 
   void stop(CoprocessorEnvironment env) throws IOException;
+
 }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java Fri Aug 30 20:31:44 2013
@@ -26,6 +26,8 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -375,8 +377,9 @@ public class ClientScanner extends Abstr
           if (values != null && values.length > 0) {
             for (Result rs : values) {
               cache.add(rs);
-              for (KeyValue kv : rs.raw()) {
-                  remainingResultSize -= kv.heapSize();
+              for (Cell kv : rs.raw()) {
+                // TODO make method in Cell or CellUtil
+                remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
               }
               countdown--;
               this.lastResult = rs;

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java Fri Aug 30 20:31:44 2013
@@ -27,6 +27,7 @@ import java.util.Map;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -149,18 +150,18 @@ public class Delete extends Mutation imp
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
-  public Delete addDeleteMarker(KeyValue kv) throws IOException {
+  public Delete addDeleteMarker(Cell kv) throws IOException {
     // TODO: Deprecate and rename 'add' so it matches how we add KVs to Puts.
-    if (!kv.isDelete()) {
+    if (!CellUtil.isDelete(kv)) {
       throw new IOException("The recently added KeyValue is not of type "
           + "delete. Rowkey: " + Bytes.toStringBinary(this.row));
     }
-    if (Bytes.compareTo(this.row, 0, row.length, kv.getBuffer(),
+    if (Bytes.compareTo(this.row, 0, row.length, kv.getRowArray(),
         kv.getRowOffset(), kv.getRowLength()) != 0) {
       throw new WrongRowIOException("The row in " + kv.toString() +
         " doesn't match the original one " +  Bytes.toStringBinary(this.row));
     }
-    byte [] family = kv.getFamily();
+    byte [] family = CellUtil.getFamilyArray(kv);
     List<Cell> list = familyMap.get(family);
     if (list == null) {
       list = new ArrayList<Cell>();

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java Fri Aug 30 20:31:44 2013
@@ -195,7 +195,7 @@ public class Get extends OperationWithAt
 
   /**
    * Apply the specified server-side filter when performing the Get.
-   * Only {@link Filter#filterKeyValue(KeyValue)} is called AFTER all tests
+   * Only {@link Filter#filterKeyValue(Cell)} is called AFTER all tests
    * for ttl, column match, deletes and max versions have been run.
    * @param filter filter to run on the server
    * @return this for invocation chaining

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java Fri Aug 30 20:31:44 2013
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -29,9 +28,9 @@ import java.util.TreeMap;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -122,7 +121,6 @@ public class Put extends Mutation implem
    * @param value column value
    * @return this
    */
-  @SuppressWarnings("unchecked")
   public Put add(byte [] family, byte [] qualifier, long ts, byte [] value) {
     if (ts < 0) {
       throw new IllegalArgumentException("Timestamp cannot be negative. ts=" + ts);
@@ -130,7 +128,7 @@ public class Put extends Mutation implem
     List<Cell> list = getCellList(family);
     KeyValue kv = createPutKeyValue(family, qualifier, ts, value);
     list.add(kv);
-    familyMap.put(kv.getFamily(), list);
+    familyMap.put(CellUtil.getFamilyArray(kv), list);
     return this;
   }
 
@@ -142,13 +140,12 @@ public class Put extends Mutation implem
    * @return this
    * @throws java.io.IOException e
    */
-  @SuppressWarnings("unchecked")
-  public Put add(KeyValue kv) throws IOException{
-    byte [] family = kv.getFamily();
+  public Put add(Cell kv) throws IOException{
+    byte [] family = CellUtil.getFamilyArray(kv);
     List<Cell> list = getCellList(family);
     //Checking that the row of the kv is the same as the put
     int res = Bytes.compareTo(this.row, 0, row.length,
-        kv.getBuffer(), kv.getRowOffset(), kv.getRowLength());
+        kv.getRowArray(), kv.getRowOffset(), kv.getRowLength());
     if (res != 0) {
       throw new WrongRowIOException("The row in " + kv.toString() +
         " doesn't match the original one " +  Bytes.toStringBinary(this.row));
@@ -245,35 +242,31 @@ public class Put extends Mutation implem
     // F F => 1
     if (!ignoreTS && !ignoreValue) {
       for (Cell cell : list) {
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        if (Arrays.equals(kv.getFamily(), family) &&
-            Arrays.equals(kv.getQualifier(), qualifier) &&
-            Arrays.equals(kv.getValue(), value) &&
-            kv.getTimestamp() == ts) {
+        if (CellUtil.matchingFamily(cell, family) &&
+            CellUtil.matchingQualifier(cell, qualifier)  &&
+            CellUtil.matchingValue(cell, value) &&
+            cell.getTimestamp() == ts) {
           return true;
         }
       }
     } else if (ignoreValue && !ignoreTS) {
       for (Cell cell : list) {
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        if (Arrays.equals(kv.getFamily(), family) && Arrays.equals(kv.getQualifier(), qualifier)
-            && kv.getTimestamp() == ts) {
+        if (CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
+            && cell.getTimestamp() == ts) {
           return true;
         }
       }
     } else if (!ignoreValue && ignoreTS) {
       for (Cell cell : list) {
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        if (Arrays.equals(kv.getFamily(), family) && Arrays.equals(kv.getQualifier(), qualifier)
-            && Arrays.equals(kv.getValue(), value)) {
+        if (CellUtil.matchingFamily(cell, family) && CellUtil.matchingQualifier(cell, qualifier)
+            && CellUtil.matchingValue(cell, value)) {
           return true;
         }
       }
     } else {
       for (Cell cell : list) {
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        if (Arrays.equals(kv.getFamily(), family) &&
-            Arrays.equals(kv.getQualifier(), qualifier)) {
+        if (CellUtil.matchingFamily(cell, family) &&
+            CellUtil.matchingQualifier(cell, qualifier)) {
           return true;
         }
       }
@@ -287,14 +280,13 @@ public class Put extends Mutation implem
    * @param family column family
    * @param qualifier column qualifier
    * @return a list of KeyValue objects with the matching family and qualifier,
-   * returns an empty list if one doesnt exist for the given family.
+   * returns an empty list if one doesn't exist for the given family.
    */
-  public List<KeyValue> get(byte[] family, byte[] qualifier) {
-    List<KeyValue> filteredList = new ArrayList<KeyValue>();
+  public List<Cell> get(byte[] family, byte[] qualifier) {
+    List<Cell> filteredList = new ArrayList<Cell>();
     for (Cell cell: getCellList(family)) {
-      KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-      if (Arrays.equals(kv.getQualifier(), qualifier)) {
-        filteredList.add(kv);
+      if (CellUtil.matchingQualifier(cell, qualifier)) {
+        filteredList.add(cell);
       }
     }
     return filteredList;

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java Fri Aug 30 20:31:44 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.CellScann
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -71,7 +72,7 @@ import org.apache.hadoop.hbase.util.Byte
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class Result implements CellScannable {
-  private KeyValue [] kvs;
+  private Cell[] cells;
   // We're not using java serialization.  Transient here is just a marker to say
   // that this is where we cache row if we're ever asked for it.
   private transient byte [] row = null;
@@ -97,10 +98,10 @@ public class Result implements CellScann
    * Instantiate a Result with the specified array of KeyValues.
    * <br><strong>Note:</strong> You must ensure that the keyvalues
    * are already sorted
-   * @param kvs array of KeyValues
+   * @param cells array of KeyValues
    */
-  public Result(KeyValue [] kvs) {
-    this.kvs = kvs;
+  public Result(Cell [] cells) {
+    this.cells = cells;
   }
 
   /**
@@ -109,9 +110,9 @@ public class Result implements CellScann
    * are already sorted
    * @param kvs List of KeyValues
    */
-  public Result(List<? extends Cell> kvs) {
+  public Result(List<Cell> kvs) {
     // TODO: Here we presume the passed in Cells are KVs.  One day this won't always be so.
-    this(kvs.toArray(new KeyValue[kvs.size()]));
+    this(kvs.toArray(new Cell[kvs.size()]));
   }
 
   /**
@@ -121,65 +122,65 @@ public class Result implements CellScann
    */
   public byte [] getRow() {
     if (this.row == null) {
-      this.row = this.kvs == null || this.kvs.length == 0? null: this.kvs[0].getRow();
+      this.row = this.cells == null || this.cells.length == 0? null: CellUtil.getRowArray(this.cells[0]);
     }
     return this.row;
   }
 
   /**
-   * Return the array of KeyValues backing this Result instance.
+   * Return the array of Cells backing this Result instance.
    *
    * The array is sorted from smallest -> largest using the
    * {@link KeyValue#COMPARATOR}.
    *
    * The array only contains what your Get or Scan specifies and no more.
    * For example if you request column "A" 1 version you will have at most 1
-   * KeyValue in the array. If you request column "A" with 2 version you will
-   * have at most 2 KeyValues, with the first one being the newer timestamp and
+   * Cell in the array. If you request column "A" with 2 version you will
+   * have at most 2 Cells, with the first one being the newer timestamp and
    * the second being the older timestamp (this is the sort order defined by
    * {@link KeyValue#COMPARATOR}).  If columns don't exist, they won't be
    * present in the result. Therefore if you ask for 1 version all columns,
-   * it is safe to iterate over this array and expect to see 1 KeyValue for
+   * it is safe to iterate over this array and expect to see 1 Cell for
    * each column and no more.
    *
    * This API is faster than using getFamilyMap() and getMap()
    *
-   * @return array of KeyValues; can be null if nothing in the result
+   * @return array of Cells; can be null if nothing in the result
    */
-  public KeyValue[] raw() {
-    return kvs;
+  public Cell[] raw() {
+    return cells;
   }
 
   /**
-   * Create a sorted list of the KeyValue's in this result.
+   * Create a sorted list of the Cell's in this result.
    *
    * Since HBase 0.20.5 this is equivalent to raw().
    *
-   * @return The sorted list of KeyValue's.
+   * @return The sorted list of Cell's.
    */
-  public List<KeyValue> list() {
+  public List<Cell> list() {
     return isEmpty()? null: Arrays.asList(raw());
   }
 
   /**
-   * Return the KeyValues for the specific column.  The KeyValues are sorted in
+   * Return the Cells for the specific column.  The Cells are sorted in
    * the {@link KeyValue#COMPARATOR} order.  That implies the first entry in
    * the list is the most recent column.  If the query (Scan or Get) only
    * requested 1 version the list will contain at most 1 entry.  If the column
    * did not exist in the result set (either the column does not exist
    * or the column was not selected in the query) the list will be empty.
    *
-   * Also see getColumnLatest which returns just a KeyValue
+   * Also see getColumnLatest which returns just a Cell
    *
    * @param family the family
    * @param qualifier
-   * @return a list of KeyValues for this column or empty list if the column
+   * @return a list of Cells for this column or empty list if the column
    * did not exist in the result set
    */
-  public List<KeyValue> getColumn(byte [] family, byte [] qualifier) {
-    List<KeyValue> result = new ArrayList<KeyValue>();
+  public List<Cell> getColumn(byte [] family, byte [] qualifier) {
+    List<Cell> result = new ArrayList<Cell>();
 
-    KeyValue [] kvs = raw();
+    Cell [] kvs = raw();
 
     if (kvs == null || kvs.length == 0) {
       return result;
@@ -190,7 +191,7 @@ public class Result implements CellScann
     }
 
     for (int i = pos ; i < kvs.length ; i++ ) {
-      KeyValue kv = kvs[i];
+      KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[i]);
       if (kv.matchingColumn(family,qualifier)) {
         result.add(kv);
       } else {
@@ -201,11 +202,11 @@ public class Result implements CellScann
     return result;
   }
 
-  protected int binarySearch(final KeyValue [] kvs,
+  protected int binarySearch(final Cell [] kvs,
                              final byte [] family,
                              final byte [] qualifier) {
-    KeyValue searchTerm =
-        KeyValue.createFirstOnRow(kvs[0].getRow(),
+    Cell searchTerm =
+        KeyValue.createFirstOnRow(CellUtil.getRowArray(kvs[0]),
             family, qualifier);
 
     // pos === ( -(insertion point) - 1)
@@ -234,7 +235,7 @@ public class Result implements CellScann
    *
    * @return the index where the value was found, or -1 otherwise
    */
-  protected int binarySearch(final KeyValue [] kvs,
+  protected int binarySearch(final Cell [] kvs,
       final byte [] family, final int foffset, final int flength,
       final byte [] qualifier, final int qoffset, final int qlength) {
 
@@ -246,8 +247,8 @@ public class Result implements CellScann
       buffer = new byte[(int) Math.ceil(keyValueSize / PAD_WIDTH) * PAD_WIDTH];
     }
 
-    KeyValue searchTerm = KeyValue.createFirstOnRow(buffer, 0,
-        kvs[0].getBuffer(), kvs[0].getRowOffset(), kvs[0].getRowLength(),
+    Cell searchTerm = KeyValue.createFirstOnRow(buffer, 0,
+        kvs[0].getRowArray(), kvs[0].getRowOffset(), kvs[0].getRowLength(),
         family, foffset, flength,
         qualifier, qoffset, qlength);
 
@@ -265,16 +266,16 @@ public class Result implements CellScann
   }
 
   /**
-   * The KeyValue for the most recent timestamp for a given column.
+   * The Cell for the most recent timestamp for a given column.
    *
    * @param family
    * @param qualifier
    *
-   * @return the KeyValue for the column, or null if no value exists in the row or none have been
+   * @return the Cell for the column, or null if no value exists in the row or none have been
    * selected in the query (Get/Scan)
    */
-  public KeyValue getColumnLatest(byte [] family, byte [] qualifier) {
-    KeyValue [] kvs = raw(); // side effect possibly.
+  public Cell getColumnLatest(byte [] family, byte [] qualifier) {
+    Cell [] kvs = raw(); // side effect possibly.
     if (kvs == null || kvs.length == 0) {
       return null;
     }
@@ -282,7 +283,7 @@ public class Result implements CellScann
     if (pos == -1) {
       return null;
     }
-    KeyValue kv = kvs[pos];
+    KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[pos]);
     if (kv.matchingColumn(family, qualifier)) {
       return kv;
     }
@@ -290,7 +291,7 @@ public class Result implements CellScann
   }
 
   /**
-   * The KeyValue for the most recent timestamp for a given column.
+   * The Cell for the most recent timestamp for a given column.
    *
    * @param family family name
    * @param foffset family offset
@@ -299,13 +300,13 @@ public class Result implements CellScann
    * @param qoffset qualifier offset
    * @param qlength qualifier length
    *
-   * @return the KeyValue for the column, or null if no value exists in the row or none have been
+   * @return the Cell for the column, or null if no value exists in the row or none have been
    * selected in the query (Get/Scan)
    */
-  public KeyValue getColumnLatest(byte [] family, int foffset, int flength,
+  public Cell getColumnLatest(byte [] family, int foffset, int flength,
       byte [] qualifier, int qoffset, int qlength) {
 
-    KeyValue [] kvs = raw(); // side effect possibly.
+    Cell [] kvs = raw(); // side effect possibly.
     if (kvs == null || kvs.length == 0) {
       return null;
     }
@@ -313,7 +314,7 @@ public class Result implements CellScann
     if (pos == -1) {
       return null;
     }
-    KeyValue kv = kvs[pos];
+    KeyValue kv = KeyValueUtil.ensureKeyValue(kvs[pos]);
     if (kv.matchingColumn(family, foffset, flength, qualifier, qoffset, qlength)) {
       return kv;
     }
@@ -327,11 +328,11 @@ public class Result implements CellScann
    * @return value of latest version of column, null if none found
    */
   public byte[] getValue(byte [] family, byte [] qualifier) {
-    KeyValue kv = getColumnLatest(family, qualifier);
+    Cell kv = getColumnLatest(family, qualifier);
     if (kv == null) {
       return null;
     }
-    return kv.getValue();
+    return CellUtil.getValueArray(kv);
   }
 
   /**
@@ -344,12 +345,12 @@ public class Result implements CellScann
    */
   public ByteBuffer getValueAsByteBuffer(byte [] family, byte [] qualifier) {
 
-    KeyValue kv = getColumnLatest(family, 0, family.length, qualifier, 0, qualifier.length);
+    Cell kv = getColumnLatest(family, 0, family.length, qualifier, 0, qualifier.length);
 
     if (kv == null) {
       return null;
     }
-    return kv.getValueAsByteBuffer();
+    return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
   }
 
   /**
@@ -367,12 +368,12 @@ public class Result implements CellScann
   public ByteBuffer getValueAsByteBuffer(byte [] family, int foffset, int flength,
       byte [] qualifier, int qoffset, int qlength) {
 
-    KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
+    Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
 
     if (kv == null) {
       return null;
     }
-    return kv.getValueAsByteBuffer();
+    return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
   }
 
   /**
@@ -413,12 +414,12 @@ public class Result implements CellScann
   public boolean loadValue(byte [] family, int foffset, int flength,
       byte [] qualifier, int qoffset, int qlength, ByteBuffer dst)
           throws BufferOverflowException {
-    KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
+    Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
 
     if (kv == null) {
       return false;
     }
-    kv.loadValue(dst);
+    dst.put(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
     return true;
   }
 
@@ -450,7 +451,7 @@ public class Result implements CellScann
   public boolean containsNonEmptyColumn(byte [] family, int foffset, int flength,
       byte [] qualifier, int qoffset, int qlength) {
 
-    KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
+    Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
 
     return (kv != null) && (kv.getValueLength() > 0);
   }
@@ -482,7 +483,7 @@ public class Result implements CellScann
    */
   public boolean containsEmptyColumn(byte [] family, int foffset, int flength,
       byte [] qualifier, int qoffset, int qlength) {
-    KeyValue kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
+    Cell kv = getColumnLatest(family, foffset, flength, qualifier, qoffset, qlength);
 
     return (kv != null) && (kv.getValueLength() == 0);
   }
@@ -496,7 +497,7 @@ public class Result implements CellScann
    * @return true if at least one value exists in the result, false if not
    */
   public boolean containsColumn(byte [] family, byte [] qualifier) {
-    KeyValue kv = getColumnLatest(family, qualifier);
+    Cell kv = getColumnLatest(family, qualifier);
     return kv != null;
   }
 
@@ -535,8 +536,8 @@ public class Result implements CellScann
       return null;
     }
     this.familyMap = new TreeMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>(Bytes.BYTES_COMPARATOR);
-    for(KeyValue kv : this.kvs) {
-      byte [] family = kv.getFamily();
+    for(Cell kv : this.cells) {
+      byte [] family = CellUtil.getFamilyArray(kv);
       NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap =
         familyMap.get(family);
       if(columnMap == null) {
@@ -544,7 +545,7 @@ public class Result implements CellScann
           (Bytes.BYTES_COMPARATOR);
         familyMap.put(family, columnMap);
       }
-      byte [] qualifier = kv.getQualifier();
+      byte [] qualifier = CellUtil.getQualifierArray(kv);
       NavigableMap<Long, byte[]> versionMap = columnMap.get(qualifier);
       if(versionMap == null) {
         versionMap = new TreeMap<Long, byte[]>(new Comparator<Long>() {
@@ -555,7 +556,7 @@ public class Result implements CellScann
         columnMap.put(qualifier, versionMap);
       }
       Long timestamp = kv.getTimestamp();
-      byte [] value = kv.getValue();
+      byte [] value = CellUtil.getValueArray(kv);
 
       versionMap.put(timestamp, value);
     }
@@ -632,22 +633,22 @@ public class Result implements CellScann
     if (isEmpty()) {
       return null;
     }
-    return kvs[0].getValue();
+    return CellUtil.getValueArray(cells[0]);
   }
 
   /**
-   * Check if the underlying KeyValue [] is empty or not
+   * Check if the underlying Cell [] is empty or not
    * @return true if empty
    */
   public boolean isEmpty() {
-    return this.kvs == null || this.kvs.length == 0;
+    return this.cells == null || this.cells.length == 0;
   }
 
   /**
-   * @return the size of the underlying KeyValue []
+   * @return the size of the underlying Cell []
    */
   public int size() {
-    return this.kvs == null? 0: this.kvs.length;
+    return this.cells == null? 0: this.cells.length;
   }
 
   /**
@@ -663,7 +664,7 @@ public class Result implements CellScann
     }
     sb.append("{");
     boolean moreThanOne = false;
-    for(KeyValue kv : this.kvs) {
+    for(Cell kv : this.cells) {
       if(moreThanOne) {
         sb.append(", ");
       } else {
@@ -691,11 +692,11 @@ public class Result implements CellScann
       throw new Exception("This row doesn't have the same number of KVs: "
           + res1.toString() + " compared to " + res2.toString());
     }
-    KeyValue[] ourKVs = res1.raw();
-    KeyValue[] replicatedKVs = res2.raw();
+    Cell[] ourKVs = res1.raw();
+    Cell[] replicatedKVs = res2.raw();
     for (int i = 0; i < res1.size(); i++) {
       if (!ourKVs[i].equals(replicatedKVs[i]) ||
-          !Bytes.equals(ourKVs[i].getValue(), replicatedKVs[i].getValue())) {
+          !Bytes.equals(CellUtil.getValueArray(ourKVs[i]), CellUtil.getValueArray(replicatedKVs[i]))) {
         throw new Exception("This result was different: "
             + res1.toString() + " compared to " + res2.toString());
       }
@@ -709,11 +710,11 @@ public class Result implements CellScann
   public void copyFrom(Result other) {
     this.row = null;
     this.familyMap = null;
-    this.kvs = other.kvs;
+    this.cells = other.cells;
   }
 
   @Override
   public CellScanner cellScanner() {
-    return CellUtil.createCellScanner(this.kvs);
+    return CellUtil.createCellScanner(this.cells);
   }
 }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java Fri Aug 30 20:31:44 2013
@@ -18,22 +18,23 @@
 
 package org.apache.hadoop.hbase.client;
 
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
+import java.io.IOException;
+import java.net.UnknownHostException;
 
 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.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@@ -46,8 +47,8 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.DNS;
 
-import java.io.IOException;
-import java.net.UnknownHostException;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
 
 /**
  * Scanner operations such as create, next, etc.
@@ -252,8 +253,9 @@ public class ScannerCallable extends Reg
     }
     long resultSize = 0;
     for (Result rr : rrs) {
-      for (KeyValue kv : rr.raw()) {
-        resultSize += kv.getLength();
+      for (Cell kv : rr.raw()) {
+        // TODO add getLength to Cell/use CellUtil#estimatedSizeOf
+        resultSize += KeyValueUtil.ensureKeyValue(kv).getLength();
       }
     }
     this.scanMetrics.countOfBytesInResults.addAndGet(resultSize);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java Fri Aug 30 20:31:44 2013
@@ -24,7 +24,8 @@ import java.math.RoundingMode;
 
 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.CellUtil;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BigDecimalMsg;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
@@ -43,12 +44,12 @@ public class BigDecimalColumnInterpreter
   EmptyMsg, BigDecimalMsg, BigDecimalMsg> {
 
   @Override
-  public BigDecimal getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
+  public BigDecimal getValue(byte[] colFamily, byte[] colQualifier, Cell kv)
       throws IOException {
-    if (kv == null || kv.getValue() == null) {
+    if (kv == null || CellUtil.getValueArray(kv) == null) {
       return null;
     }
-    return Bytes.toBigDecimal(kv.getValue()).setScale(2, RoundingMode.HALF_EVEN);
+    return Bytes.toBigDecimal(CellUtil.getValueArray(kv)).setScale(2, RoundingMode.HALF_EVEN);
   }
 
   @Override

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java Fri Aug 30 20:31:44 2013
@@ -18,16 +18,16 @@
  */
 package org.apache.hadoop.hbase.client.coprocessor;
 
+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.coprocessor.ColumnInterpreter;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LongMsg;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-
 /**
  * a concrete column interpreter implementation. The cell value is a Long value
  * and its promoted data type is also a Long value. For computing aggregation
@@ -41,11 +41,11 @@ import java.io.IOException;
 public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
                  EmptyMsg, LongMsg, LongMsg> {
 
-  public Long getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
+  public Long getValue(byte[] colFamily, byte[] colQualifier, Cell kv)
       throws IOException {
     if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG)
       return null;
-    return Bytes.toLong(kv.getBuffer(), kv.getValueOffset());
+    return Bytes.toLong(kv.getValueArray(), kv.getValueOffset());
   }
 
    @Override

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java Fri Aug 30 20:31:44 2013
@@ -19,13 +19,16 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import com.google.protobuf.Message;
+import java.io.IOException;
+
 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.coprocessor.LongColumnInterpreter;
 
-import java.io.IOException;
+import com.google.protobuf.Message;
 
 /**
  * Defines how value for specific column is interpreted and provides utility
@@ -59,14 +62,32 @@ public abstract class ColumnInterpreter<
 Q extends Message, R extends Message> {
 
   /**
+   * TODO: when removing {@link #getValue(byte[], byte[], KeyValue)}, this method should be made abstract
+   * 
    * @param colFamily
    * @param colQualifier
-   * @param kv
+   * @param c
    * @return value of type T
    * @throws IOException
    */
-  public abstract T getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
-      throws IOException;
+  public T getValue(byte[] colFamily, byte[] colQualifier, Cell c)
+      throws IOException {
+    // call the deprecated method for compatiblity.
+    KeyValue kv = KeyValueUtil.ensureKeyValue(c);
+    return getValue(colFamily, colQualifier, kv);
+  }
+
+  /**
+   * This method used to be abstract, and is preserved for compatibility and easy of conversion
+   * from 0.94->0.96.
+   *
+   * Please override {@link #getValue(byte[], byte[], Cell)} instead.
+   */
+  @Deprecated
+  public T getValue(byte[] colFamily, byte[] colQualifier, KeyValue kv)
+      throws IOException {
+    return null;
+  }
 
   /**
    * @param l1

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java Fri Aug 30 20:31:44 2013
@@ -19,15 +19,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;
 
 /**
  * Simple filter that returns first N columns on row only.
@@ -56,7 +57,7 @@ public class ColumnCountGetFilter extend
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell v) {
     this.count++;
     return filterAllRemaining() ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL;
   }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java Fri Aug 30 20:31:44 2013
@@ -23,6 +23,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;
@@ -103,13 +104,13 @@ public class ColumnPaginationFilter exte
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v)
+  public ReturnCode filterKeyValue(Cell v)
   {
     if (columnOffset != null) {
       if (count >= limit) {
         return ReturnCode.NEXT_ROW;
       }
-      byte[] buffer = v.getBuffer();
+      byte[] buffer = v.getQualifierArray();
       if (buffer == null) {
         return ReturnCode.SEEK_NEXT_USING_HINT;
       }
@@ -141,9 +142,10 @@ public class ColumnPaginationFilter exte
     }
   }
 
-  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(), columnOffset, 0, columnOffset.length);
   }
 

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java Fri Aug 30 20:31:44 2013
@@ -24,6 +24,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;
@@ -50,11 +51,11 @@ public class ColumnPrefixFilter extends 
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue kv) {
-    if (this.prefix == null || kv.getBuffer() == null) {
+  public ReturnCode filterKeyValue(Cell kv) {
+    if (this.prefix == null || kv.getQualifierArray() == null) {
       return ReturnCode.INCLUDE;
     } else {
-      return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+      return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
     }
   }
 
@@ -127,9 +128,10 @@ public class ColumnPrefixFilter extends 
     return Bytes.equals(this.getPrefix(), other.getPrefix());
   }
 
-  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(), prefix, 0, prefix.length);
   }
 

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java Fri Aug 30 20:31:44 2013
@@ -26,6 +26,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;
@@ -114,8 +115,9 @@ public class ColumnRangeFilter extends F
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue kv) {
-    byte[] buffer = kv.getBuffer();
+  public ReturnCode filterKeyValue(Cell kv) {
+    // TODO have a column compare method in Cell
+    byte[] buffer = kv.getQualifierArray();
     int qualifierOffset = kv.getQualifierOffset();
     int qualifierLength = kv.getQualifierLength();
     int cmpMin = 1;
@@ -213,9 +215,9 @@ public class ColumnRangeFilter extends F
   }
 
   @Override
-  public KeyValue getNextKeyHint(KeyValue kv) {
-    return KeyValue.createFirstOnRow(kv.getBuffer(), kv.getRowOffset(), kv
-        .getRowLength(), kv.getBuffer(), kv.getFamilyOffset(), kv
+  public Cell getNextCellHint(Cell kv) {
+    return KeyValue.createFirstOnRow(kv.getRowArray(), kv.getRowOffset(), kv
+        .getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv
         .getFamilyLength(), this.minColumn, 0, len(this.minColumn));
 
   }

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java Fri Aug 30 20:31:44 2013
@@ -120,7 +120,8 @@ public abstract class CompareFilter exte
     }
   }
 
-  public static ArrayList extractArguments(ArrayList<byte []> filterArguments) {
+  // returns an array of heterogeneous objects
+  public static ArrayList<Object> extractArguments(ArrayList<byte []> filterArguments) {
     Preconditions.checkArgument(filterArguments.size() == 2,
                                 "Expected 2 but got: %s", filterArguments.size());
     CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(0));
@@ -135,7 +136,7 @@ public abstract class CompareFilter exte
                                             " can only be used with EQUAL and NOT_EQUAL");
       }
     }
-    ArrayList arguments = new ArrayList();
+    ArrayList<Object> arguments = new ArrayList<Object>();
     arguments.add(compareOp);
     arguments.add(comparator);
     return arguments;

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java Fri Aug 30 20:31:44 2013
@@ -18,23 +18,26 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
 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.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter for adding inter-column timestamp matching
@@ -77,9 +80,9 @@ public class DependentColumnFilter exten
   
   /**
    * Constructor for DependentColumn filter.
-   * Keyvalues where a keyvalue from target column 
-   * with the same timestamp do not exist will be dropped. 
-   * 
+   * Cells where a Cell from target column
+   * with the same timestamp do not exist will be dropped.
+   *
    * @param family name of target column family
    * @param qualifier name of column qualifier
    */
@@ -89,12 +92,12 @@ public class DependentColumnFilter exten
   
   /**
    * Constructor for DependentColumn filter.
-   * Keyvalues where a keyvalue from target column 
-   * with the same timestamp do not exist will be dropped. 
-   * 
+   * Cells where a Cell from target column
+   * with the same timestamp do not exist will be dropped.
+   *
    * @param family name of dependent column family
    * @param qualifier name of dependent qualifier
-   * @param dropDependentColumn whether the dependent columns keyvalues should be discarded
+   * @param dropDependentColumn whether the dependent columns Cells should be discarded
    */
   public DependentColumnFilter(final byte [] family, final byte [] qualifier,
       final boolean dropDependentColumn) {
@@ -132,7 +135,9 @@ public class DependentColumnFilter exten
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell c) {
+    // TODO make matching Column a cell method or CellUtil method.
+    KeyValue v = KeyValueUtil.ensureKeyValue(c);
     // Check if the column and qualifier match
   	if (!v.matchingColumn(this.columnFamily, this.columnQualifier)) {
         // include non-matches for the time being, they'll be discarded afterwards
@@ -140,7 +145,7 @@ public class DependentColumnFilter exten
   	}
     // If it doesn't pass the op, skip it
     if (comparator != null
-        && doCompare(compareOp, comparator, v.getBuffer(), v.getValueOffset(),
+        && doCompare(compareOp, comparator, v.getValueArray(), v.getValueOffset(),
             v.getValueLength()))
       return ReturnCode.SKIP;
 	
@@ -152,9 +157,9 @@ public class DependentColumnFilter exten
   }
 
   @Override
-  public void filterRow(List<KeyValue> kvs) {
-    Iterator<KeyValue> it = kvs.iterator();
-    KeyValue kv;
+  public void filterRowCells(List<Cell> kvs) {
+    Iterator<? extends Cell> it = kvs.iterator();
+    Cell kv;
     while(it.hasNext()) {
       kv = it.next();
       if(!stampSet.contains(kv.getTimestamp())) {

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java Fri Aug 30 20:31:44 2013
@@ -19,16 +19,17 @@
 
 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.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 family. It takes an
@@ -59,10 +60,10 @@ public class FamilyFilter extends Compar
   }
 
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) {
+  public ReturnCode filterKeyValue(Cell v) {
     int familyLength = v.getFamilyLength();
     if (familyLength > 0) {
-      if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+      if (doCompare(this.compareOp, this.comparator, v.getFamilyArray(),
           v.getFamilyOffset(), familyLength)) {
         return ReturnCode.SKIP;
       }
@@ -71,7 +72,7 @@ public class FamilyFilter extends Compar
   }
 
   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 FamilyFilter(compareOp, comparator);

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Fri Aug 30 20:31:44 2013
@@ -24,6 +24,7 @@ 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.exceptions.DeserializationException;
 
@@ -35,11 +36,11 @@ import org.apache.hadoop.hbase.exception
  *   <li> {@link #reset()} : reset the filter state before filtering a new row. </li>
  *   <li> {@link #filterAllRemaining()}: true means row scan is over; false means keep going. </li>
  *   <li> {@link #filterRowKey(byte[],int,int)}: true means drop this row; false means include.</li>
- *   <li> {@link #filterKeyValue(KeyValue)}: decides whether to include or exclude this KeyValue.
+ *   <li> {@link #filterKeyValue(Cell)}: decides whether to include or exclude this KeyValue.
  *        See {@link ReturnCode}. </li>
  *   <li> {@link #transform(KeyValue)}: if the KeyValue is included, let the filter transform the
  *        KeyValue. </li>
- *   <li> {@link #filterRow(List)}: allows direct modification of the final list to be submitted
+ *   <li> {@link #filterRowCells(List)}: allows direct modification of the final list to be submitted
  *   <li> {@link #filterRow()}: last chance to drop entire row based on the sequence of
  *        filter calls. Eg: filter a row if it doesn't contain a specified column. </li>
  * </ul>
@@ -67,7 +68,7 @@ public abstract class Filter {
 
   /**
    * Filters a row based on the row key. If this returns true, the entire row will be excluded. If
-   * false, each KeyValue in the row will be passed to {@link #filterKeyValue(KeyValue)} below.
+   * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below.
    * 
    * Concrete implementers can signal a failure condition in their code by throwing an
    * {@link IOException}.
@@ -103,16 +104,16 @@ public abstract class Filter {
    * Concrete implementers can signal a failure condition in their code by throwing an
    * {@link IOException}.
    * 
-   * @param v the KeyValue in question
+   * @param v the Cell in question
    * @return code as described below
    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
    * @see Filter.ReturnCode
    */
-  abstract public ReturnCode filterKeyValue(final KeyValue v) throws IOException;
+  abstract public ReturnCode filterKeyValue(final Cell v) throws IOException;
 
   /**
-   * Give the filter a chance to transform the passed KeyValue. If the KeyValue is changed a new
-   * KeyValue object must be returned.
+   * Give the filter a chance to transform the passed KeyValue. If the Cell is changed a new
+   * Cell object must be returned.
    * 
    * @see org.apache.hadoop.hbase.KeyValue#shallowCopy()
    *      The transformed KeyValue is what is eventually returned to the client. Most filters will
@@ -127,22 +128,30 @@ public abstract class Filter {
    * @return the changed KeyValue
    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
    */
-  abstract public KeyValue transform(final KeyValue v) throws IOException;
+  abstract public Cell transformCell(final Cell v) throws IOException;
 
   /**
+   * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
+   * This is for transition from 0.94 -> 0.96
+   **/
+  @Deprecated // use Cell transformCell(final Cell)
+  abstract public KeyValue transform(final KeyValue currentKV) throws IOException;
+ 
+  
+  /**
    * Return codes for filterValue().
    */
   public enum ReturnCode {
     /**
-     * Include the KeyValue
+     * Include the Cell
      */
     INCLUDE,
     /**
-     * Include the KeyValue and seek to the next column skipping older versions.
+     * Include the Cell and seek to the next column skipping older versions.
      */
     INCLUDE_AND_NEXT_COL,
     /**
-     * Skip this KeyValue
+     * Skip this Cell
      */
     SKIP,
     /**
@@ -161,14 +170,21 @@ public abstract class Filter {
 }
 
   /**
-   * Chance to alter the list of keyvalues to be submitted. Modifications to the list will carry on
+   * Chance to alter the list of Cells to be submitted. Modifications to the list will carry on
    * 
    * Concrete implementers can signal a failure condition in their code by throwing an
    * {@link IOException}.
    * 
-   * @param kvs the list of keyvalues to be filtered
+   * @param kvs the list of Cells to be filtered
    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
    */
+  abstract public void filterRowCells(List<Cell> kvs) throws IOException;
+
+  /**
+   * WARNING: please to not override this method.  Instead override {@link #filterRowCells(List)}.
+   * This is for transition from 0.94 -> 0.96
+   **/
+  @Deprecated
   abstract public void filterRow(List<KeyValue> kvs) throws IOException;
 
   /**
@@ -180,7 +196,7 @@ public abstract class Filter {
   abstract public boolean hasFilterRow();
 
   /**
-   * Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)} calls. The filter
+   * Last chance to veto row based on previous {@link #filterKeyValue(Cell)} calls. The filter
    * needs to retain state then return a particular value for this call if they wish to exclude a
    * row if a certain column is missing (for example).
    * 
@@ -192,6 +208,9 @@ public abstract class Filter {
    */
   abstract public boolean filterRow() throws IOException;
 
+  @Deprecated // use Cell GetNextKeyHint(final Cell)
+  abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException;
+
   /**
    * If the filter returns the match code SEEK_NEXT_USING_HINT, then it should also tell which is
    * the next key it must seek to. After receiving the match code SEEK_NEXT_USING_HINT, the
@@ -204,7 +223,7 @@ public abstract class Filter {
    *         seek to next.
    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
    */
-  abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException;
+  abstract public Cell getNextCellHint(final Cell currentKV) throws IOException;
 
   /**
    * Check that given column family is essential for filter to check row. Most filters always return

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java Fri Aug 30 20:31:44 2013
@@ -5,7 +5,8 @@
  * distributed with this work for additional information
  * regarding copyright ownership.  The ASF licenses this file
  * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance                                                                       with the License.  You may obtain a copy of the License at
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
  *     http://www.apache.org/licenses/LICENSE-2.0
  *
@@ -24,7 +25,9 @@ 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;
 
 /**
  * Abstract base class to help you implement new Filters.  Common "ignore" or NOOP type
@@ -72,12 +75,12 @@ public abstract class FilterBase extends
 
   /**
    * Filters that dont filter by key value can inherit this implementation that
-   * includes all KeyValues.
+   * includes all Cells.
    *
    * @inheritDoc
    */
   @Override
-  public ReturnCode filterKeyValue(KeyValue ignored) throws IOException {
+  public ReturnCode filterKeyValue(Cell ignored) throws IOException {
     return ReturnCode.INCLUDE;
   }
 
@@ -87,22 +90,54 @@ public abstract class FilterBase extends
    * @inheritDoc
    */
   @Override
-  public KeyValue transform(KeyValue v) throws IOException {
-    return 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));
   }
 
   /**
-   * Filters that never filter by modifying the returned List of KeyValues can
+   * 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 currentKV;
+  }
+
+  /**
+   * Filters that never filter by modifying the returned List of Cells can
    * inherit this implementation that does nothing.
    *
    * @inheritDoc
    */
   @Override
-  public void filterRow(List<KeyValue> ignored) throws IOException {
+  public void filterRowCells(List<Cell> ignored) 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.
+    List<KeyValue> kvs = new ArrayList<KeyValue>(ignored.size());
+    for (Cell c : ignored) {
+      kvs.add(KeyValueUtil.ensureKeyValue(c));
+    }
+    filterRow(kvs);
+    ignored.clear();
+    ignored.addAll(kvs);
+  }
+
+  /**
+   * 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 {
   }
 
   /**
-   * Fitlers that never filter by modifying the returned List of KeyValues can
+   * Fitlers that never filter by modifying the returned List of Cells can
    * inherit this implementation that does nothing.
    *
    * @inheritDoc
@@ -114,7 +149,7 @@ public abstract class FilterBase extends
 
   /**
    * Filters that never filter by rows based on previously gathered state from
-   * {@link #filterKeyValue(KeyValue)} can inherit this implementation that
+   * {@link #filterKeyValue(Cell)} can inherit this implementation that
    * never filters a row.
    *
    * @inheritDoc
@@ -125,13 +160,24 @@ public abstract class FilterBase extends
   }
 
   /**
+   * This method is deprecated and you should override Cell getNextKeyHint(Cell) instead.
+   */
+  @Override
+  @Deprecated
+  public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
+    return null;
+  }
+  
+  /**
    * Filters that are not sure which key must be next seeked to, can inherit
-   * this implementation that, by default, returns a null KeyValue.
+   * this implementation that, by default, returns a null Cell.
    *
    * @inheritDoc
    */
-  public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
-    return null;
+  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 getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV));
   }
 
   /**

Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java Fri Aug 30 20:31:44 2013
@@ -25,7 +25,9 @@ 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;
@@ -51,7 +53,7 @@ import com.google.protobuf.InvalidProtoc
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class FilterList extends Filter {
+final public class FilterList extends Filter {
   /** set operator */
   public static enum Operator {
     /** !AND */
@@ -65,17 +67,17 @@ public class FilterList extends Filter {
   private List<Filter> filters = new ArrayList<Filter>();
   private Filter seekHintFilter = null;
 
-  /** Reference KeyValue used by {@link #transform(KeyValue)} for validation purpose. */
-  private KeyValue referenceKV = null;
+  /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */
+  private Cell referenceKV = null;
 
   /**
-   * When filtering a given KeyValue in {@link #filterKeyValue(KeyValue)},
-   * this stores the transformed KeyValue to be returned by {@link #transform(KeyValue)}.
+   * When filtering a given Cell in {@link #filterKeyValue(Cell)},
+   * this stores the transformed Cell to be returned by {@link #transformCell(Cell)}.
    *
-   * Individual filters transformation are applied only when the filter includes the KeyValue.
+   * Individual filters transformation are applied only when the filter includes the Cell.
    * Transformations are composed in the order specified by {@link #filters}.
    */
-  private KeyValue transformedKV = null;
+  private Cell transformedKV = null;
 
   /**
    * Constructor that takes a set of {@link Filter}s. The default operator
@@ -202,21 +204,35 @@ public class FilterList extends Filter {
   }
 
   @Override
+  public Cell transformCell(Cell v) throws IOException {
+    return transform(KeyValueUtil.ensureKeyValue(v));
+  }
+
+  /**
+   * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
+   *
+   * When removing this, its body should be placed in transformCell.
+   *
+   * This is for transition from 0.94 -> 0.96
+   */
+  @Deprecated
+  @Override
   public KeyValue transform(KeyValue v) throws IOException {
     // transform() is expected to follow an inclusive filterKeyValue() immediately:
     if (!v.equals(this.referenceKV)) {
       throw new IllegalStateException(
-          "Reference KeyValue: " + this.referenceKV + " does not match: " + v);
+          "Reference Cell: " + this.referenceKV + " does not match: " + v);
      }
-    return this.transformedKV;
+    return KeyValueUtil.ensureKeyValue(this.transformedKV);
   }
 
+  
   @Override
-  public ReturnCode filterKeyValue(KeyValue v) throws IOException {
+  public ReturnCode filterKeyValue(Cell v) throws IOException {
     this.referenceKV = v;
 
-    // Accumulates successive transformation of every filter that includes the KeyValue:
-    KeyValue transformed = v;
+    // Accumulates successive transformation of every filter that includes the Cell:
+    Cell transformed = v;
 
     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
         ReturnCode.SKIP: ReturnCode.INCLUDE;
@@ -231,7 +247,7 @@ public class FilterList extends Filter {
         case INCLUDE_AND_NEXT_COL:
           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
         case INCLUDE:
-          transformed = filter.transform(transformed);
+          transformed = filter.transformCell(transformed);
           continue;
         case SEEK_NEXT_USING_HINT:
           seekHintFilter = filter;
@@ -249,11 +265,11 @@ public class FilterList extends Filter {
           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
             rc = ReturnCode.INCLUDE;
           }
-          transformed = filter.transform(transformed);
+          transformed = filter.transformCell(transformed);
           break;
         case INCLUDE_AND_NEXT_COL:
           rc = ReturnCode.INCLUDE_AND_NEXT_COL;
-          transformed = filter.transform(transformed);
+          transformed = filter.transformCell(transformed);
           // must continue here to evaluate all filters
           break;
         case NEXT_ROW:
@@ -270,19 +286,56 @@ public class FilterList extends Filter {
       }
     }
 
-    // Save the transformed KeyValue for transform():
+    // Save the transformed Cell for transform():
     this.transformedKV = transformed;
 
     return rc;
   }
 
+  /**
+   * Filters that never filter by modifying the returned List of Cells can
+   * inherit this implementation that does nothing.
+   *
+   * @inheritDoc
+   */
+  @Override
+  public void filterRowCells(List<Cell> ignored) 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.
+    List<KeyValue> kvs = new ArrayList<KeyValue>(ignored.size());
+    for (Cell c : ignored) {
+      kvs.add(KeyValueUtil.ensureKeyValue(c));
+    }
+    filterRow(kvs);
+    ignored.clear();
+    ignored.addAll(kvs);
+  }
+
+  /**
+   * 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 {
+    // when removing this, this body should be in filterRowCells
+
+    // convert to List<Cell> and call the new interface (this will call 0.96-style
+    // #filterRowCells(List<Cell>) which may delegate to legacy #filterRow(List<KV>) 
+    List<Cell> cells = new ArrayList<Cell>(kvs.size());
+    cells.addAll(kvs);
     for (Filter filter : filters) {
-      filter.filterRow(kvs);
+      filter.filterRowCells(cells); 
     }
-  }
 
+    // convert results into kvs
+    kvs.clear();
+    for (Cell c : cells) {
+      kvs.add(KeyValueUtil.ensureKeyValue(c));
+    }
+  }
+  
   @Override
   public boolean hasFilterRow() {
     for (Filter filter : filters) {
@@ -364,16 +417,22 @@ public class FilterList extends Filter {
   }
 
   @Override
+  @Deprecated
   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
-    KeyValue keyHint = null;
+    return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
+  }
+
+  @Override
+  public Cell getNextCellHint(Cell currentKV) throws IOException {
+    Cell keyHint = null;
     if (operator == Operator.MUST_PASS_ALL) {
-      keyHint = seekHintFilter.getNextKeyHint(currentKV);
+      keyHint = seekHintFilter.getNextCellHint(currentKV);
       return keyHint;
     }
 
     // If any condition can pass, we need to keep the min hint
     for (Filter filter : filters) {
-      KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
+      Cell curKeyHint = filter.getNextCellHint(currentKV);
       if (curKeyHint == null) {
         // If we ever don't have a hint and this is must-pass-one, then no hint
         return null;



Mime
View raw message