You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2011/10/29 00:29:50 UTC

svn commit: r1190640 - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/client/coprocessor/ src/main/java/org/apache/hadoop/hbase/filter/ src/main/java/org/apache/hadoop/hbase/regionserver/ src/main/java/org/apache/hadoop/hbase/replication/reg...

Author: larsh
Date: Fri Oct 28 22:29:49 2011
New Revision: 1190640

URL: http://svn.apache.org/viewvc?rev=1190640&view=rev
Log:
HBASE-4691  Remove more unnecessary byte[] copies from KeyValues

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1190640&r1=1190639&r2=1190640&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Fri Oct 28 22:29:49 2011
@@ -26,6 +26,7 @@ Release 0.93.0 - Unreleased
    HBASE-4418  Show all the hbase configuration in the web ui
    HBASE-4489  Better key splitting in RegionSplitter
    HBASE-4626  Filters unnecessarily copy byte arrays (Lars H)
+   HBASE-4691  Remove more unnecessary byte[] copies from KeyValues (Lars H)
 
   BUG FIXES
    HBASE-4488  Store could miss rows during flush (Lars H via jgray)

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java?rev=1190640&r1=1190639&r2=1190640&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java Fri Oct 28 22:29:49 2011
@@ -39,9 +39,9 @@ public class LongColumnInterpreter imple
 
   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

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java?rev=1190640&r1=1190639&r2=1190640&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java Fri Oct 28 22:29:49 2011
@@ -139,9 +139,11 @@ public class DependentColumnFilter exten
         // 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) {

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1190640&r1=1190639&r2=1190640&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Oct 28 22:29:49 2011
@@ -2026,7 +2026,9 @@ public class HRegion implements HeapSize
         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;
@@ -3896,7 +3898,8 @@ public class HRegion implements HeapSize
             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++;
             }
 

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java?rev=1190640&r1=1190639&r2=1190640&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java Fri Oct 28 22:29:49 2011
@@ -22,9 +22,7 @@ package org.apache.hadoop.hbase.replicat
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
@@ -62,8 +60,6 @@ public class ReplicationSink {
   private final Configuration conf;
   // Pool used to replicated
   private final HTablePool pool;
-  // Chain to pull on when we want all to stop.
-  private final Stoppable stopper;
   private final ReplicationSinkMetrics metrics;
 
   /**
@@ -78,7 +74,6 @@ public class ReplicationSink {
     this.conf = conf;
     this.pool = new HTablePool(this.conf,
         conf.getInt("replication.sink.htablepool.capacity", 10));
-    this.stopper = stopper;
     this.metrics = new ReplicationSinkMetrics();
   }
 
@@ -126,17 +121,17 @@ public class ReplicationSink {
           }
           // 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);
         }
@@ -172,7 +167,7 @@ public class ReplicationSink {
       this.metrics.appliedOpsRate.inc(puts.size());
     } finally {
       if (table != null) {
-        this.pool.putTable(table);
+        table.close();
       }
     }
   }
@@ -191,7 +186,7 @@ public class ReplicationSink {
       this.metrics.appliedOpsRate.inc(1);
     } finally {
       if (table != null) {
-        this.pool.putTable(table);
+        table.close();
       }
     }
   }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java?rev=1190640&r1=1190639&r2=1190640&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java Fri Oct 28 22:29:49 2011
@@ -52,7 +52,7 @@ implements ColumnAggregationProtocol {
         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();