You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/12/21 05:29:44 UTC

svn commit: r1424808 [1/3] - in /hbase/trunk: hbase-protocol/src/main/protobuf/ hbase-server/src/main/java/org/apache/hadoop/hbase/client/ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ hbase-server/src/main/java/org/apache/hadoop/hbas...

Author: stack
Date: Fri Dec 21 04:29:43 2012
New Revision: 1424808

URL: http://svn.apache.org/viewvc?rev=1424808&view=rev
Log:
HBASE-6789 Convert test CoprocessorProtocol implementations to protocol buffer services

Added:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationProtos.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/PingProtos.java
    hbase/trunk/hbase-server/src/test/protobuf/ColumnAggregationProtocol.proto
    hbase/trunk/hbase-server/src/test/protobuf/PingProtocol.proto
Removed:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationProtocol.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/GenericEndpoint.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/GenericProtocol.java
Modified:
    hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseEndpointCoprocessor.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java

Modified: hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto (original)
+++ hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto Fri Dec 21 04:29:43 2012
@@ -283,12 +283,14 @@ message Exec {
    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(
    * Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
    * method before they are available.
+   * @deprecated Use CoprocessorService going forward
  */
 message ExecCoprocessorRequest {
   required RegionSpecifier region = 1;
   required Exec call = 2;
 }
 
+// @deprecated Use CoprocessorService going forward
 message ExecCoprocessorResponse {
   required NameBytesPair value = 1;
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Fri Dec 21 04:29:43 2012
@@ -320,6 +320,7 @@ public interface HConnection extends Abo
    * @param <T> the protocol interface type
    * @param <R> the callable's return type
    * @throws IOException
+   * @deprecated CoprocessorProtocol replaced by CoprocessorService calls.
    */
   public <T extends CoprocessorProtocol,R> void processExecs(
       final Class<T> protocol,

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java Fri Dec 21 04:29:43 2012
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.Pool
 
 /**
  * A simple pool of HTable instances.
- * 
+ *
  * Each HTablePool acts as a pool for all tables. To use, instantiate an
  * HTablePool and use {@link #getTable(String)} to get an HTable from the pool.
  *
@@ -51,12 +51,12 @@ import org.apache.hadoop.hbase.util.Pool
  * Once you are done with it, close your instance of {@link HTableInterface}
  * by calling {@link HTableInterface#close()} rather than returning the tables
  * to the pool with (deprecated) {@link #putTable(HTableInterface)}.
- * 
+ *
  * <p>
  * A pool can be created with a <i>maxSize</i> which defines the most HTable
  * references that will ever be retained for each table. Otherwise the default
  * is {@link Integer#MAX_VALUE}.
- * 
+ *
  * <p>
  * Pool will manage its own connections to the cluster. See
  * {@link HConnectionManager}.
@@ -79,7 +79,7 @@ public class HTablePool implements Close
 
   /**
    * Constructor to set maximum versions and use the specified configuration.
-   * 
+   *
    * @param config
    *          configuration
    * @param maxSize
@@ -92,7 +92,7 @@ public class HTablePool implements Close
   /**
    * Constructor to set maximum versions and use the specified configuration and
    * table factory.
-   * 
+   *
    * @param config
    *          configuration
    * @param maxSize
@@ -108,7 +108,7 @@ public class HTablePool implements Close
   /**
    * Constructor to set maximum versions and use the specified configuration and
    * pool type.
-   * 
+   *
    * @param config
    *          configuration
    * @param maxSize
@@ -128,7 +128,7 @@ public class HTablePool implements Close
    * {@link PoolType#Reusable} and {@link PoolType#ThreadLocal}. If the pool
    * type is null or not one of those two values, then it will default to
    * {@link PoolType#Reusable}.
-   * 
+   *
    * @param config
    *          configuration
    * @param maxSize
@@ -168,7 +168,7 @@ public class HTablePool implements Close
    * Get a reference to the specified table from the pool.
    * <p>
    * <p/>
-   * 
+   *
    * @param tableName
    *          table name
    * @return a reference to the specified table
@@ -186,9 +186,9 @@ public class HTablePool implements Close
   /**
    * Get a reference to the specified table from the pool.
    * <p>
-   * 
+   *
    * Create a new one if one is not available.
-   * 
+   *
    * @param tableName
    *          table name
    * @return a reference to the specified table
@@ -206,9 +206,9 @@ public class HTablePool implements Close
   /**
    * Get a reference to the specified table from the pool.
    * <p>
-   * 
+   *
    * Create a new one if one is not available.
-   * 
+   *
    * @param tableName
    *          table name
    * @return a reference to the specified table
@@ -222,7 +222,7 @@ public class HTablePool implements Close
   /**
    * This method is not needed anymore, clients should call
    * HTableInterface.close() rather than returning the tables to the pool
-   * 
+   *
    * @param table
    *          the proxy table user got from pool
    * @deprecated
@@ -248,10 +248,10 @@ public class HTablePool implements Close
   /**
    * Puts the specified HTable back into the pool.
    * <p>
-   * 
+   *
    * If the pool already contains <i>maxSize</i> references to the table, then
    * the table instance gets closed after flushing buffered edits.
-   * 
+   *
    * @param table
    *          table
    */
@@ -279,7 +279,7 @@ public class HTablePool implements Close
    * Note: this is a 'shutdown' of the given table pool and different from
    * {@link #putTable(HTableInterface)}, that is used to return the table
    * instance to the pool for future re-use.
-   * 
+   *
    * @param tableName
    */
   public void closeTablePool(final String tableName) throws IOException {
@@ -294,7 +294,7 @@ public class HTablePool implements Close
 
   /**
    * See {@link #closeTablePool(String)}.
-   * 
+   *
    * @param tableName
    */
   public void closeTablePool(final byte[] tableName) throws IOException {
@@ -457,7 +457,7 @@ public class HTablePool implements Close
 
     /**
      * Returns the actual table back to the pool
-     * 
+     *
      * @throws IOException
      */
     public void close() throws IOException {
@@ -475,12 +475,14 @@ public class HTablePool implements Close
     }
 
     @Override
+    @Deprecated
     public <T extends CoprocessorProtocol> T coprocessorProxy(
         Class<T> protocol, byte[] row) {
       return table.coprocessorProxy(protocol, row);
     }
 
     @Override
+    @Deprecated
     public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
         Class<T> protocol, byte[] startKey, byte[] endKey,
         Batch.Call<T, R> callable) throws IOException, Throwable {
@@ -488,6 +490,7 @@ public class HTablePool implements Close
     }
 
     @Override
+    @Deprecated
     public <T extends CoprocessorProtocol, R> void coprocessorExec(
         Class<T> protocol, byte[] startKey, byte[] endKey,
         Batch.Call<T, R> callable, Batch.Callback<R> callback)
@@ -521,7 +524,7 @@ public class HTablePool implements Close
 
     /**
      * Expose the wrapped HTable to tests in the same package
-     * 
+     *
      * @return wrapped htable
      */
     HTableInterface getWrappedTable() {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseEndpointCoprocessor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseEndpointCoprocessor.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseEndpointCoprocessor.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseEndpointCoprocessor.java Fri Dec 21 04:29:43 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.ipc.Versi
  * However you still can just implement the interface CoprocessorProtocol
  * and Coprocessor to develop an Endpoint. But you won't be able to access
  * the region related resource, i.e., CoprocessorEnvironment.
+ * @deprecated CoprocessorProtocol is going away in 0.96
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Fri Dec 21 04:29:43 2012
@@ -131,7 +131,7 @@ public abstract class CoprocessorHost<E 
   protected void loadSystemCoprocessors(Configuration conf, String confKey) {
     Class<?> implClass = null;
 
-    // load default coprocessors from configure file    
+    // load default coprocessors from configure file
     String[] defaultCPClasses = conf.getStrings(confKey);
     if (defaultCPClasses == null || defaultCPClasses.length == 0)
       return;
@@ -175,7 +175,7 @@ public abstract class CoprocessorHost<E 
   public E load(Path path, String className, int priority,
       Configuration conf) throws IOException {
     Class<?> implClass = null;
-    LOG.debug("Loading coprocessor class " + className + " with path " + 
+    LOG.debug("Loading coprocessor class " + className + " with path " +
         path + " and priority " + priority);
 
     ClassLoader cl = null;
@@ -587,6 +587,7 @@ public abstract class CoprocessorHost<E 
       }
 
       @Override
+      @Deprecated
       public <T extends CoprocessorProtocol, R> void coprocessorExec(Class<T> protocol,
           byte[] startKey, byte[] endKey, Batch.Call<T, R> callable,
           Batch.Callback<R> callback) throws IOException, Throwable {
@@ -594,6 +595,7 @@ public abstract class CoprocessorHost<E 
       }
 
       @Override
+    @Deprecated
       public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
           Class<T> protocol, byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
           throws IOException, Throwable {
@@ -601,6 +603,7 @@ public abstract class CoprocessorHost<E 
       }
 
       @Override
+    @Deprecated
       public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
           byte[] row) {
         return table.coprocessorProxy(protocol, row);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Fri Dec 21 04:29:43 2012
@@ -241,6 +241,7 @@ public final class ProtobufUtil {
    * @return the converted client Exec
    */
   @SuppressWarnings("unchecked")
+  @Deprecated
   public static Exec toExec(
       final ClientProtos.Exec proto) throws IOException {
     byte[] row = proto.getRow().toByteArray();
@@ -542,10 +543,10 @@ public final class ProtobufUtil {
 
   /**
    * Convert a MutateRequest to Mutation
-   * 
+   *
    * @param proto the protocol buffer Mutate to convert
    * @return the converted Mutation
-   * @throws IOException 
+   * @throws IOException
    */
   public static Mutation toMutation(final Mutate proto) throws IOException {
     MutateType type = proto.getMutateType();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Dec 21 04:29:43 2012
@@ -2608,7 +2608,7 @@ public class HRegion implements HeapSize
     p.setWriteToWAL(true);
     doBatchMutate(p, lid);
   }
- 
+
   /**
    * Atomically apply the given map of family->edits to the memstore.
    * This handles the consistency control on its own, but the caller
@@ -2802,7 +2802,7 @@ public class HRegion implements HeapSize
       }
     }
     long seqid = minSeqIdForTheRegion;
-    
+
     NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(fs, regiondir);
     if (files == null || files.isEmpty()) return seqid;
 
@@ -3420,7 +3420,7 @@ public class HRegion implements HeapSize
       } else {
         this.filter = null;
       }
-      
+
       this.batch = scan.getBatch();
       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
         this.stopRow = null;
@@ -3574,7 +3574,7 @@ public class HRegion implements HeapSize
           if (filter != null && filter.hasFilterRow()) {
             filter.filterRow(results);
           }
-          
+
           return false;
         } else if (filterRowKey(currentRow, offset, length)) {
           nextRow(currentRow, offset, length);
@@ -3627,7 +3627,7 @@ public class HRegion implements HeapSize
     protected void nextRow(byte [] currentRow, int offset, short length) throws IOException {
       KeyValue next;
       while((next = this.storeHeap.peek()) != null && next.matchingRow(currentRow, offset, length)) {
-        this.storeHeap.next(MOCKED_LIST);       
+        this.storeHeap.next(MOCKED_LIST);
       }
       results.clear();
       resetFilters();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Dec 21 04:29:43 2012
@@ -105,7 +105,6 @@ import org.apache.hadoop.hbase.filter.Co
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
 import org.apache.hadoop.hbase.ipc.HBaseRPC;
 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServer;
@@ -344,10 +343,10 @@ public class  HRegionServer implements C
 
   /** region server process name */
   public static final String REGIONSERVER = "regionserver";
-  
+
   /** region server configuration name */
   public static final String REGIONSERVER_CONF = "regionserver_conf";
-  
+
   /*
    * Space is reserved in HRS constructor and then released when aborting to
    * recover from an OOME. See HBASE-706. TODO: Make this percentage of the heap
@@ -432,7 +431,7 @@ public class  HRegionServer implements C
    * The reference to the QosFunction
    */
   private final QosFunction qosFunction;
-  
+
   private RegionServerCoprocessorHost rsHost;
 
   /**
@@ -627,7 +626,7 @@ public class  HRegionServer implements C
             new HashMap<Class<? extends Message>, Method>());
       }
       if (methodMap.get("getRegion") == null) {
-        methodMap.put("getRegion", 
+        methodMap.put("getRegion",
             new HashMap<Class<? extends Message>, Method>());
       }
       for (Class<? extends Message> cls : knownArgumentClasses) {
@@ -1397,7 +1396,7 @@ public class  HRegionServer implements C
     // Instantiate replication manager if replication enabled.  Pass it the
     // log directories.
     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
-    
+
     return instantiateHLog(rootDir, logName);
   }
 
@@ -2108,7 +2107,7 @@ public class  HRegionServer implements C
   public ZooKeeperWatcher getZooKeeperWatcher() {
     return this.zooKeeper;
   }
-  
+
   public RegionServerCoprocessorHost getCoprocessorHost(){
     return this.rsHost;
   }
@@ -3392,7 +3391,7 @@ public class  HRegionServer implements C
         checkIfRegionInTransition(region.getEncodedNameAsBytes(), OPEN);
         HRegion onlineRegion = getFromOnlineRegions(region.getEncodedName());
         if (null != onlineRegion) {
-          //Check if the region can actually be opened. 
+          //Check if the region can actually be opened.
           if( onlineRegion.getCoprocessorHost() != null){
             onlineRegion.getCoprocessorHost().preOpen();
           }
@@ -3486,7 +3485,7 @@ public class  HRegionServer implements C
         }
         checkIfRegionInTransition(encodedName, CLOSE);
       }
-           
+
       requestCount.increment();
       LOG.info("Received close region: " + region.getRegionNameAsString() +
         ". Version of ZK closing node:" + versionOfClosingNode +
@@ -3978,7 +3977,7 @@ public class  HRegionServer implements C
   private String getMyEphemeralNodePath() {
     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
   }
-  
+
   /**
    * Holder class which holds the RegionScanner and nextCallSeq together.
    */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java Fri Dec 21 04:29:43 2012
@@ -430,7 +430,7 @@ public class RegionCoprocessorHost
    * Called prior to rewriting the store files selected for compaction
    * @param store the store being compacted
    * @param scanner the scanner used to read store data during compaction
-   * @throws IOException 
+   * @throws IOException
    */
   public InternalScanner preCompact(HStore store, InternalScanner scanner) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
@@ -503,7 +503,7 @@ public class RegionCoprocessorHost
 
   /**
    * Invoked before a memstore flush
-   * @throws IOException 
+   * @throws IOException
    */
   public void preFlush() throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
@@ -607,7 +607,7 @@ public class RegionCoprocessorHost
       }
     }
   }
-  
+
   /**
    * Invoked just before a split
    * @throws IOException
@@ -633,7 +633,7 @@ public class RegionCoprocessorHost
    * Invoked just after a split
    * @param l the new left-hand daughter region
    * @param r the new right-hand daughter region
-   * @throws IOException 
+   * @throws IOException
    */
   public void postSplit(HRegion l, HRegion r) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
@@ -651,7 +651,7 @@ public class RegionCoprocessorHost
       }
     }
   }
-  
+
   /**
    * Invoked just before the rollback of a failed split is started
    * @throws IOException
@@ -672,7 +672,7 @@ public class RegionCoprocessorHost
       }
     }
   }
-  
+
   /**
    * Invoked just after the rollback of a failed split is done
    * @throws IOException
@@ -693,7 +693,7 @@ public class RegionCoprocessorHost
       }
     }
   }
-  
+
   /**
    * Invoked after a split is completed irrespective of a failure or success.
    * @throws IOException

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Fri Dec 21 04:29:43 2012
@@ -82,7 +82,7 @@ public class RemoteHTable implements HTa
   final long sleepTime;
 
   @SuppressWarnings("rawtypes")
-  protected String buildRowSpec(final byte[] row, final Map familyMap, 
+  protected String buildRowSpec(final byte[] row, final Map familyMap,
       final long startTime, final long endTime, final int maxVersions) {
     StringBuffer sb = new StringBuffer();
     sb.append('/');
@@ -174,7 +174,7 @@ public class RemoteHTable implements HTa
         byte[][] split = KeyValue.parseColumn(cell.getColumn());
         byte[] column = split[0];
         byte[] qualifier = split.length > 1 ? split[1] : null;
-        kvs.add(new KeyValue(row.getKey(), column, qualifier, 
+        kvs.add(new KeyValue(row.getKey(), column, qualifier,
           cell.getTimestamp(), cell.getValue()));
       }
       results.add(new Result(kvs));
@@ -252,7 +252,7 @@ public class RemoteHTable implements HTa
         TableSchemaModel schema = new TableSchemaModel();
         schema.getObjectFromMessage(response.getBody());
         return schema.getTableDescriptor();
-      case 509: 
+      case 509:
         try {
           Thread.sleep(sleepTime);
         } catch (InterruptedException e) { }
@@ -529,7 +529,7 @@ public class RemoteHTable implements HTa
       }
       return results[0];
     }
-    
+
     class Iter implements Iterator<Result> {
 
       Result cache;
@@ -563,7 +563,7 @@ public class RemoteHTable implements HTa
       public void remove() {
         throw new RuntimeException("remove() not supported");
       }
-      
+
     }
 
     @Override
@@ -726,6 +726,7 @@ public class RemoteHTable implements HTa
   }
 
   @Override
+  @Deprecated
   public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
       byte[] row) {
     throw new
@@ -733,6 +734,7 @@ public class RemoteHTable implements HTa
   }
 
   @Override
+  @Deprecated
   public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
       Class<T> protocol, byte[] startKey, byte[] endKey,
       Batch.Call<T, R> callable)
@@ -741,6 +743,7 @@ public class RemoteHTable implements HTa
   }
 
   @Override
+  @Deprecated
   public <T extends CoprocessorProtocol, R> void coprocessorExec(
       Class<T> protocol, byte[] startKey, byte[] endKey,
       Batch.Call<T, R> callable, Batch.Callback<R> callback)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java Fri Dec 21 04:29:43 2012
@@ -22,29 +22,67 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.ColumnAggregationService;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
+import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+
 
 /**
  * The aggregation implementation at a region.
  */
-public class ColumnAggregationEndpoint extends BaseEndpointCoprocessor
-implements ColumnAggregationProtocol {
+public class ColumnAggregationEndpoint extends ColumnAggregationService
+implements Coprocessor, CoprocessorService {
+  static final Log LOG = LogFactory.getLog(ColumnAggregationEndpoint.class);
+  private RegionCoprocessorEnvironment env = null;
+
+  @Override
+  public Service getService() {
+    return this;
+  }
+
+  @Override
+  public void start(CoprocessorEnvironment env) throws IOException {
+    if (env instanceof RegionCoprocessorEnvironment) {
+      this.env = (RegionCoprocessorEnvironment)env;
+      return;
+    }
+    throw new CoprocessorException("Must be loaded on a table region!");
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment env) throws IOException {
+    // Nothing to do.
+  }
 
   @Override
-  public long sum(byte[] family, byte[] qualifier)
-  throws IOException {
+  public void sum(RpcController controller, SumRequest request, RpcCallback<SumResponse> done) {
     // aggregate at each region
     Scan scan = new Scan();
-    scan.addColumn(family, qualifier);
+    // Family is required in pb. Qualifier is not.
+    byte [] family = request.getFamily().toByteArray();
+    byte [] qualifier = request.hasQualifier()? request.getQualifier().toByteArray(): null;
+    if (request.hasQualifier()) {
+      scan.addColumn(family, qualifier);
+    } else {
+      scan.addFamily(family);
+    }
     int sumResult = 0;
-
-    InternalScanner scanner = ((RegionCoprocessorEnvironment)getEnvironment())
-        .getRegion().getScanner(scan);
+    InternalScanner scanner = null;
     try {
+      scanner = this.env.getRegion().getScanner(scan);
       List<KeyValue> curVals = new ArrayList<KeyValue>();
       boolean hasMore = false;
       do {
@@ -56,9 +94,22 @@ implements ColumnAggregationProtocol {
           }
         }
       } while (hasMore);
+    } catch (IOException e) {
+      ResponseConverter.setControllerException(controller, e);
+      // Set result to -1 to indicate error.
+      sumResult = -1;
+      LOG.info("Setting sum result to -1 to indicate error", e);
     } finally {
-      scanner.close();
+      if (scanner != null) {
+        try {
+          scanner.close();
+        } catch (IOException e) {
+          ResponseConverter.setControllerException(controller, e);
+          sumResult = -1;
+          LOG.info("Setting sum result to -1 to indicate error", e);
+        }
+      }
     }
-    return sumResult;
+    done.run(SumResponse.newBuilder().setSum(sumResult).build());
   }
-}
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java Fri Dec 21 04:29:43 2012
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.TestServerCustomProtocol;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -64,7 +65,8 @@ public class TestClassLoading {
   static final String cpNameInvalid = "TestCPInvalid";
 
   private static Class<?> regionCoprocessor1 = ColumnAggregationEndpoint.class;
-  private static Class<?> regionCoprocessor2 = GenericEndpoint.class;
+  // TOOD: Fix the import of this handler.  It is coming in from a package that is far away.
+  private static Class<?> regionCoprocessor2 = TestServerCustomProtocol.PingHandler.class;
   private static Class<?> regionServerCoprocessor = SampleRegionWALObserver.class;
   private static Class<?> masterCoprocessor = BaseMasterObserver.class;
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java Fri Dec 21 04:29:43 2012
@@ -18,34 +18,44 @@
  */
 package org.apache.hadoop.hbase.coprocessor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
-import com.google.protobuf.RpcController;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 
 /**
  * TestEndpoint: test cases to verify coprocessor Endpoint
@@ -59,9 +69,6 @@ public class TestCoprocessorEndpoint {
   private static final byte[] TEST_QUALIFIER = Bytes.toBytes("TestQualifier");
   private static byte[] ROW = Bytes.toBytes("testRow");
   
-  private static final String protocolName =  "org.apache.hadoop.hbase.CustomProtocol";
-  private static final String methodName = "myFunc";
-
   private static final int ROWSIZE = 20;
   private static final int rowSeperator1 = 5;
   private static final int rowSeperator2 = 12;
@@ -75,7 +82,6 @@ public class TestCoprocessorEndpoint {
     Configuration conf = util.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         org.apache.hadoop.hbase.coprocessor.ColumnAggregationEndpoint.class.getName(),
-        org.apache.hadoop.hbase.coprocessor.GenericEndpoint.class.getName(),
         ProtobufCoprocessorService.class.getName());
     conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
         ProtobufCoprocessorService.class.getName());
@@ -101,51 +107,34 @@ public class TestCoprocessorEndpoint {
     util.shutdownMiniCluster();
   }
 
-  @Test
-  public void testGeneric() throws Throwable {
-    HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-    GenericProtocol protocol = table.coprocessorProxy(GenericProtocol.class,
-                                                      Bytes.toBytes("testRow"));
-    String workResult1 = protocol.doWork("foo");
-    assertEquals("foo", workResult1);
-    byte[] workResult2 = protocol.doWork(new byte[]{1});
-    assertArrayEquals(new byte[]{1}, workResult2);
-    byte workResult3 = protocol.doWork((byte)1);
-    assertEquals((byte)1, workResult3);
-    char workResult4 = protocol.doWork('c');
-    assertEquals('c', workResult4);
-    boolean workResult5 = protocol.doWork(true);
-    assertEquals(true, workResult5);
-    short workResult6 = protocol.doWork((short)1);
-    assertEquals((short)1, workResult6);
-    int workResult7 = protocol.doWork(5);
-    assertEquals(5, workResult7);
-    long workResult8 = protocol.doWork(5l);
-    assertEquals(5l, workResult8);
-    double workResult9 = protocol.doWork(6d);
-    assertEquals(6d, workResult9, 0.01);
-    float workResult10 = protocol.doWork(6f);
-    assertEquals(6f, workResult10, 0.01);
-    Text workResult11 = protocol.doWork(new Text("foo"));
-    assertEquals(new Text("foo"), workResult11);
-    table.close();
+  private Map<byte [], Long> sum(final HTable table, final byte [] family,
+      final byte [] qualifier, final byte [] start, final byte [] end)
+  throws ServiceException, Throwable {
+    return table.coprocessorService(ColumnAggregationProtos.ColumnAggregationService.class,
+        start, end,
+      new Batch.Call<ColumnAggregationProtos.ColumnAggregationService, Long>() {
+        @Override
+        public Long call(ColumnAggregationProtos.ColumnAggregationService instance)
+        throws IOException {
+          BlockingRpcCallback<ColumnAggregationProtos.SumResponse> rpcCallback =
+              new BlockingRpcCallback<ColumnAggregationProtos.SumResponse>();
+          ColumnAggregationProtos.SumRequest.Builder builder =
+            ColumnAggregationProtos.SumRequest.newBuilder();
+          builder.setFamily(ByteString.copyFrom(family));
+          if (qualifier != null && qualifier.length > 0) {
+            builder.setQualifier(ByteString.copyFrom(qualifier));
+          }
+          instance.sum(null, builder.build(), rpcCallback);
+          return rpcCallback.get().getSum();
+        }
+      });
   }
 
   @Test
   public void testAggregation() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-    Map<byte[], Long> results;
-
-    // scan: for all regions
-    results = table
-        .coprocessorExec(ColumnAggregationProtocol.class,
-                         ROWS[0], ROWS[ROWS.length-1],
-                         new Batch.Call<ColumnAggregationProtocol, Long>() {
-                           public Long call(ColumnAggregationProtocol instance)
-                               throws IOException {
-                             return instance.sum(TEST_FAMILY, TEST_QUALIFIER);
-                           }
-                         });
+    Map<byte[], Long> results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
+      ROWS[0], ROWS[ROWS.length-1]);
     int sumResult = 0;
     int expectedResult = 0;
     for (Map.Entry<byte[], Long> e : results.entrySet()) {
@@ -160,15 +149,8 @@ public class TestCoprocessorEndpoint {
     results.clear();
 
     // scan: for region 2 and region 3
-    results = table
-        .coprocessorExec(ColumnAggregationProtocol.class,
-                         ROWS[rowSeperator1], ROWS[ROWS.length-1],
-                         new Batch.Call<ColumnAggregationProtocol, Long>() {
-                           public Long call(ColumnAggregationProtocol instance)
-                               throws IOException {
-                             return instance.sum(TEST_FAMILY, TEST_QUALIFIER);
-                           }
-                         });
+    results = sum(table, TEST_FAMILY, TEST_QUALIFIER,
+      ROWS[rowSeperator1], ROWS[ROWS.length-1]);
     sumResult = 0;
     expectedResult = 0;
     for (Map.Entry<byte[], Long> e : results.entrySet()) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java?rev=1424808&r1=1424807&r2=1424808&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java Fri Dec 21 04:29:43 2012
@@ -19,11 +19,22 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
-import java.io.InterruptedIOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -35,8 +46,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-
 /**
  * Tests unhandled exceptions thrown by coprocessors running on master.
  * Expected result is that the master will abort with an informative

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationProtos.java?rev=1424808&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationProtos.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationProtos.java Fri Dec 21 04:29:43 2012
@@ -0,0 +1,1120 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: ColumnAggregationProtocol.proto
+
+package org.apache.hadoop.hbase.coprocessor.protobuf.generated;
+
+public final class ColumnAggregationProtos {
+  private ColumnAggregationProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface SumRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required bytes family = 1;
+    boolean hasFamily();
+    com.google.protobuf.ByteString getFamily();
+    
+    // optional bytes qualifier = 2;
+    boolean hasQualifier();
+    com.google.protobuf.ByteString getQualifier();
+  }
+  public static final class SumRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements SumRequestOrBuilder {
+    // Use SumRequest.newBuilder() to construct.
+    private SumRequest(Builder builder) {
+      super(builder);
+    }
+    private SumRequest(boolean noInit) {}
+    
+    private static final SumRequest defaultInstance;
+    public static SumRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public SumRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumRequest_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumRequest_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required bytes family = 1;
+    public static final int FAMILY_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString family_;
+    public boolean hasFamily() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public com.google.protobuf.ByteString getFamily() {
+      return family_;
+    }
+    
+    // optional bytes qualifier = 2;
+    public static final int QUALIFIER_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString qualifier_;
+    public boolean hasQualifier() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public com.google.protobuf.ByteString getQualifier() {
+      return qualifier_;
+    }
+    
+    private void initFields() {
+      family_ = com.google.protobuf.ByteString.EMPTY;
+      qualifier_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasFamily()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, family_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, qualifier_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, family_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, qualifier_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest other = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest) obj;
+      
+      boolean result = true;
+      result = result && (hasFamily() == other.hasFamily());
+      if (hasFamily()) {
+        result = result && getFamily()
+            .equals(other.getFamily());
+      }
+      result = result && (hasQualifier() == other.hasQualifier());
+      if (hasQualifier()) {
+        result = result && getQualifier()
+            .equals(other.getQualifier());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasFamily()) {
+        hash = (37 * hash) + FAMILY_FIELD_NUMBER;
+        hash = (53 * hash) + getFamily().hashCode();
+      }
+      if (hasQualifier()) {
+        hash = (37 * hash) + QUALIFIER_FIELD_NUMBER;
+        hash = (53 * hash) + getQualifier().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumRequest_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumRequest_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        family_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest build() {
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest buildPartial() {
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest result = new org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.family_ = family_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.qualifier_ = qualifier_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest other) {
+        if (other == org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.getDefaultInstance()) return this;
+        if (other.hasFamily()) {
+          setFamily(other.getFamily());
+        }
+        if (other.hasQualifier()) {
+          setQualifier(other.getQualifier());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasFamily()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              family_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              qualifier_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required bytes family = 1;
+      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasFamily() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public com.google.protobuf.ByteString getFamily() {
+        return family_;
+      }
+      public Builder setFamily(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        family_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearFamily() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        family_ = getDefaultInstance().getFamily();
+        onChanged();
+        return this;
+      }
+      
+      // optional bytes qualifier = 2;
+      private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasQualifier() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public com.google.protobuf.ByteString getQualifier() {
+        return qualifier_;
+      }
+      public Builder setQualifier(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        qualifier_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearQualifier() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        qualifier_ = getDefaultInstance().getQualifier();
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:SumRequest)
+    }
+    
+    static {
+      defaultInstance = new SumRequest(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:SumRequest)
+  }
+  
+  public interface SumResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required int64 sum = 1;
+    boolean hasSum();
+    long getSum();
+  }
+  public static final class SumResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements SumResponseOrBuilder {
+    // Use SumResponse.newBuilder() to construct.
+    private SumResponse(Builder builder) {
+      super(builder);
+    }
+    private SumResponse(boolean noInit) {}
+    
+    private static final SumResponse defaultInstance;
+    public static SumResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public SumResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumResponse_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumResponse_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required int64 sum = 1;
+    public static final int SUM_FIELD_NUMBER = 1;
+    private long sum_;
+    public boolean hasSum() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getSum() {
+      return sum_;
+    }
+    
+    private void initFields() {
+      sum_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasSum()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt64(1, sum_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(1, sum_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse other = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse) obj;
+      
+      boolean result = true;
+      result = result && (hasSum() == other.hasSum());
+      if (hasSum()) {
+        result = result && (getSum()
+            == other.getSum());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasSum()) {
+        hash = (37 * hash) + SUM_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSum());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumResponse_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.internal_static_SumResponse_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        sum_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse build() {
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse buildPartial() {
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse result = new org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.sum_ = sum_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse other) {
+        if (other == org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance()) return this;
+        if (other.hasSum()) {
+          setSum(other.getSum());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasSum()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              sum_ = input.readInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required int64 sum = 1;
+      private long sum_ ;
+      public boolean hasSum() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getSum() {
+        return sum_;
+      }
+      public Builder setSum(long value) {
+        bitField0_ |= 0x00000001;
+        sum_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearSum() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        sum_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:SumResponse)
+    }
+    
+    static {
+      defaultInstance = new SumResponse(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:SumResponse)
+  }
+  
+  public static abstract class ColumnAggregationService
+      implements com.google.protobuf.Service {
+    protected ColumnAggregationService() {}
+    
+    public interface Interface {
+      public abstract void sum(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse> done);
+      
+    }
+    
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new ColumnAggregationService() {
+        @java.lang.Override
+        public  void sum(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse> done) {
+          impl.sum(controller, request, done);
+        }
+        
+      };
+    }
+    
+    public static com.google.protobuf.BlockingService
+        newReflectiveBlockingService(final BlockingInterface impl) {
+      return new com.google.protobuf.BlockingService() {
+        public final com.google.protobuf.Descriptors.ServiceDescriptor
+            getDescriptorForType() {
+          return getDescriptor();
+        }
+        
+        public final com.google.protobuf.Message callBlockingMethod(
+            com.google.protobuf.Descriptors.MethodDescriptor method,
+            com.google.protobuf.RpcController controller,
+            com.google.protobuf.Message request)
+            throws com.google.protobuf.ServiceException {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.callBlockingMethod() given method descriptor for " +
+              "wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return impl.sum(controller, (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest)request);
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+        public final com.google.protobuf.Message
+            getRequestPrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getRequestPrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+        public final com.google.protobuf.Message
+            getResponsePrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getResponsePrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+        
+      };
+    }
+    
+    public abstract void sum(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse> done);
+    
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.getDescriptor().getServices().get(0);
+    }
+    public final com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    
+    public final void callMethod(
+        com.google.protobuf.Descriptors.MethodDescriptor method,
+        com.google.protobuf.RpcController controller,
+        com.google.protobuf.Message request,
+        com.google.protobuf.RpcCallback<
+          com.google.protobuf.Message> done) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.callMethod() given method descriptor for wrong " +
+          "service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          this.sum(controller, (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse>specializeCallback(
+              done));
+          return;
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+    
+    public final com.google.protobuf.Message
+        getRequestPrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getRequestPrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+    
+    public final com.google.protobuf.Message
+        getResponsePrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getResponsePrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+    
+    public static Stub newStub(
+        com.google.protobuf.RpcChannel channel) {
+      return new Stub(channel);
+    }
+    
+    public static final class Stub extends org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.ColumnAggregationService implements Interface {
+      private Stub(com.google.protobuf.RpcChannel channel) {
+        this.channel = channel;
+      }
+      
+      private final com.google.protobuf.RpcChannel channel;
+      
+      public com.google.protobuf.RpcChannel getChannel() {
+        return channel;
+      }
+      
+      public  void sum(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.class,
+            org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance()));
+      }
+    }
+    
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+    
+    public interface BlockingInterface {
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse sum(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest request)
+          throws com.google.protobuf.ServiceException;
+    }
+    
+    private static final class BlockingStub implements BlockingInterface {
+      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+        this.channel = channel;
+      }
+      
+      private final com.google.protobuf.BlockingRpcChannel channel;
+      
+      public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse sum(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.getDefaultInstance());
+      }
+      
+    }
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_SumRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_SumRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_SumResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_SumResponse_fieldAccessorTable;
+  
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\037ColumnAggregationProtocol.proto\"/\n\nSum" +
+      "Request\022\016\n\006family\030\001 \002(\014\022\021\n\tqualifier\030\002 \001" +
+      "(\014\"\032\n\013SumResponse\022\013\n\003sum\030\001 \002(\0032<\n\030Column" +
+      "AggregationService\022 \n\003sum\022\013.SumRequest\032\014" +
+      ".SumResponseBW\n6org.apache.hadoop.hbase." +
+      "coprocessor.protobuf.generatedB\027ColumnAg" +
+      "gregationProtos\210\001\001\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_SumRequest_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_SumRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_SumRequest_descriptor,
+              new java.lang.String[] { "Family", "Qualifier", },
+              org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.class,
+              org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest.Builder.class);
+          internal_static_SumResponse_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_SumResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_SumResponse_descriptor,
+              new java.lang.String[] { "Sum", },
+              org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.class,
+              org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse.Builder.class);
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+  
+  // @@protoc_insertion_point(outer_class_scope)
+}