hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1527789 [1/3] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-protocol/ hbase-protocol...
Date Mon, 30 Sep 2013 21:27:34 GMT
Author: stack
Date: Mon Sep 30 21:27:33 2013
New Revision: 1527789

URL: http://svn.apache.org/r1527789
Log:
HBASE-9612 Ability to batch edits destined to different regions -- REVERT

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
    hbase/trunk/hbase-protocol/README.txt
    hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
    hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java
    hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto
    hbase/trunk/hbase-protocol/src/main/protobuf/RPC.proto
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.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/wal/WALEditsReplaySink.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
Mon Sep 30 21:27:33 2013
@@ -171,7 +171,6 @@ public class ClientSmallScanner extends 
         ScanResponse response = null;
         PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
         try {
-          controller.setPriority(getTableName());
           response = getStub().scan(controller, request);
           return ResponseConverter.getResults(controller.cellScanner(),
               response);

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
Mon Sep 30 21:27:33 2013
@@ -643,7 +643,6 @@ public class HBaseAdmin implements Abort
             .getServerName());
         PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
         try {
-          controller.setPriority(tableName);
           ScanResponse response = server.scan(controller, request);
           values = ResponseConverter.getResults(controller.cellScanner(), response);
         } catch (ServiceException se) {

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Mon
Sep 30 21:27:33 2013
@@ -961,8 +961,6 @@ public class HTable implements HTableInt
         try {
           MultiRequest request = RequestConverter.buildMultiRequest(
             getLocation().getRegionInfo().getRegionName(), rm);
-          PayloadCarryingRpcController pcrc = new PayloadCarryingRpcController();
-          pcrc.setPriority(tableName);
           getStub().multi(null, request);
         } catch (ServiceException se) {
           throw ProtobufUtil.getRemoteException(se);
@@ -989,7 +987,6 @@ public class HTable implements HTableInt
             MutateRequest request = RequestConverter.buildMutateRequest(
               getLocation().getRegionInfo().getRegionName(), append);
             PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
-            rpcController.setPriority(getTableName());
             MutateResponse response = getStub().mutate(rpcController, request);
             if (!response.hasResult()) return null;
             return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
@@ -1016,10 +1013,9 @@ public class HTable implements HTableInt
         try {
           MutateRequest request = RequestConverter.buildMutateRequest(
             getLocation().getRegionInfo().getRegionName(), increment);
-            PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
-            rpcController.setPriority(getTableName());
-            MutateResponse response = getStub().mutate(rpcController, request);
-            return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
+            PayloadCarryingRpcController rpcContoller = new PayloadCarryingRpcController();
+            MutateResponse response = getStub().mutate(rpcContoller, request);
+            return ProtobufUtil.toResult(response.getResult(), rpcContoller.cellScanner());
           } catch (ServiceException se) {
             throw ProtobufUtil.getRemoteException(se);
           }
@@ -1078,7 +1074,6 @@ public class HTable implements HTableInt
               getLocation().getRegionInfo().getRegionName(), row, family,
               qualifier, amount, durability);
             PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
-            rpcController.setPriority(getTableName());
             MutateResponse response = getStub().mutate(rpcController, request);
             Result result =
               ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
Mon Sep 30 21:27:33 2013
@@ -24,8 +24,8 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
@@ -84,11 +84,8 @@ class MultiServerCallable<R> extends Reg
               multiRequest = RequestConverter.buildMultiRequest(regionName, rms);
             }
             // Carry the cells if any over the proxy/pb Service interface using the payload
-            // carrying rpc controller.  Also set priority on this controller so available
down
-            // in RpcClient when we go to craft the request header.
-            PayloadCarryingRpcController pcrc = new PayloadCarryingRpcController(cells);
-            pcrc.setPriority(getTableName());
-            getStub().multi(pcrc, multiRequest);
+            // carrying rpc controller.
+            getStub().multi(new PayloadCarryingRpcController(cells), multiRequest);
             // This multi call does not return results.
             response.add(regionName, action.getOriginalIndex(), Result.EMPTY_RESULT);
           } catch (ServiceException se) {
@@ -116,7 +113,6 @@ class MultiServerCallable<R> extends Reg
           // Controller optionally carries cell data over the proxy/service boundary and
also
           // optionally ferries cell response data back out again.
           PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
-          controller.setPriority(getTableName());
           ClientProtos.MultiResponse responseProto = getStub().multi(controller, multiRequest);
           results = ResponseConverter.getResults(responseProto, controller.cellScanner());
         } catch (ServiceException se) {

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
Mon Sep 30 21:27:33 2013
@@ -163,7 +163,6 @@ public class ScannerCallable extends Reg
           ScanResponse response = null;
           PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
           try {
-            controller.setPriority(getTableName());
             response = getStub().scan(controller, request);
             // Client and RS maintain a nextCallSeq number during the scan. Every next()
call
             // from client to server will increment this number in both sides. Client passes
this

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
Mon Sep 30 21:27:33 2013
@@ -23,8 +23,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
+
 
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
@@ -37,15 +36,6 @@ import com.google.protobuf.RpcController
  */
 @InterfaceAudience.Private
 public class PayloadCarryingRpcController implements RpcController, CellScannable {
-  /**
-   * Priority to set on this request.  Set it here in controller so available composing the
-   * request.  This is the ordained way of setting priorities going forward.  We will be
-   * undoing the old annotation-based mechanism.
-   */
-  // Currently only multi call makes use of this.  Eventually this should be only way to
set
-  // priority.
-  private int priority = 0;
-
   // TODO: Fill out the rest of this class methods rather than return UnsupportedOperationException
 
   /**
@@ -113,26 +103,4 @@ public class PayloadCarryingRpcControlle
   public void startCancel() {
     throw new UnsupportedOperationException();
   }
-
-  /**
-   * @param priority Priority for this request; should fall roughly in the range
-   * {@link HConstants#NORMAL_QOS} to {@link HConstants#HIGH_QOS}
-   */
-  public void setPriority(int priority) {
-    this.priority = priority;
-  }
-
-  /**
-   * @param tn Set priority based off the table we are going against.
-   */
-  public void setPriority(final TableName tn) {
-    this.priority = tn != null && tn.isSystemTable()? HConstants.HIGH_QOS: HConstants.NORMAL_QOS;
-  }
-
-  /**
-   * @return The priority of this request
-   */
-  public int getPriority() {
-    return priority;
-  }
 }
\ No newline at end of file

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java Mon
Sep 30 21:27:33 2013
@@ -1002,10 +1002,9 @@ public class RpcClient {
      * Note: this is not called from the Connection thread, but by other
      * threads.
      * @param call
-     * @param priority
      * @see #readResponse()
      */
-    protected void writeRequest(Call call, final int priority) {
+    protected void writeRequest(Call call) {
       if (shouldCloseConnection.get()) return;
       try {
         RequestHeader.Builder builder = RequestHeader.newBuilder();
@@ -1023,8 +1022,6 @@ public class RpcClient {
           cellBlockBuilder.setLength(cellBlock.limit());
           builder.setCellBlockMeta(cellBlockBuilder.build());
         }
-        // Only pass priority if there one.  Let zero be same as no priority.
-        if (priority != 0) builder.setPriority(priority);
         //noinspection SynchronizeOnNonFinalField
         RequestHeader header = builder.build();
         synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
@@ -1383,12 +1380,6 @@ public class RpcClient {
     }
   }
 
-  Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
-      Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout)
-  throws InterruptedException, IOException {
-    return call(md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS);
-  }
-
   /** Make a call, passing <code>param</code>, to the IPC server running at
    * <code>address</code> which is servicing the <code>protocol</code>
protocol,
    * with the <code>ticket</code> credentials, returning the value.
@@ -1409,12 +1400,12 @@ public class RpcClient {
    */
   Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
       Message returnType, User ticket, InetSocketAddress addr,
-      int rpcTimeout, int priority)
+      int rpcTimeout)
   throws InterruptedException, IOException {
     Call call = new Call(md, param, cells, returnType);
     Connection connection =
       getConnection(ticket, call, addr, rpcTimeout, this.codec, this.compressor);
-    connection.writeRequest(call, priority);                 // send the parameter
+    connection.writeRequest(call);                 // send the parameter
     boolean interrupted = false;
     //noinspection SynchronizationOnLocalVariableOrMethodParameter
     synchronized (call) {
@@ -1641,8 +1632,7 @@ public class RpcClient {
     }
     Pair<Message, CellScanner> val = null;
     try {
-      val = call(md, param, cells, returnType, ticket, isa, rpcTimeout,
-        pcrc != null? pcrc.getPriority(): HConstants.NORMAL_QOS);
+      val = call(md, param, cells, returnType, ticket, isa, rpcTimeout);
       if (pcrc != null) {
         // Shove the results into controller so can be carried across the proxy/pb service
void.
         if (val.getSecond() != null) pcrc.setCellScanner(val.getSecond());

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
Mon Sep 30 21:27:33 2013
@@ -2257,11 +2257,11 @@ public final class ProtobufUtil {
           ", row=" + getStringForByteString(r.getGet().getRow());
     } else if (m instanceof ClientProtos.MultiRequest) {
       ClientProtos.MultiRequest r = (ClientProtos.MultiRequest) m;
-      ClientProtos.RegionMutation rm = r.getRegionMutationList().get(0);
-      return "region= " + getStringForByteString(rm.getRegion().getValue()) +
-          ", for " + r.getRegionMutationCount() +
-          " actions and 1st row key=" + getStringForByteString(rm.getMutationCount() >
0?
-          rm.getMutation(0).getRow(): ByteString.EMPTY);
+      ClientProtos.MultiAction action = r.getActionList().get(0);
+      return "region= " + getStringForByteString(r.getRegion().getValue()) +
+          ", for " + r.getActionCount() +
+          " actions and 1st row key=" + getStringForByteString(action.hasMutation() ?
+          action.getMutation().getRow() : action.getGet().getRow());
     } else if (m instanceof ClientProtos.MutateRequest) {
       ClientProtos.MutateRequest r = (ClientProtos.MutateRequest) m;
       return "region= " + getStringForByteString(r.getRegion().getValue()) +

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
Mon Sep 30 21:27:33 2013
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
@@ -70,7 +71,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionMutation;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
@@ -365,11 +365,10 @@ public final class RequestConverter {
    * @return a multi request
    * @throws IOException
    */
-  public static MultiRequest buildMultiRequest(final byte [] regionName,
+  public static MultiRequest buildMultiRequest(final byte[] regionName,
       final RowMutations rowMutations)
   throws IOException {
-    RegionMutation.Builder builder =
-        getRegionMutationBuilderWithRegionAndAtomicSet(regionName, true);
+    MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName,
true);
     for (Mutation mutation: rowMutations.getMutations()) {
       MutationType mutateType = null;
       if (mutation instanceof Put) {
@@ -381,9 +380,9 @@ public final class RequestConverter {
           mutation.getClass().getName());
       }
       MutationProto mp = ProtobufUtil.toMutation(mutateType, mutation);
-      builder.addMutation(mp);
+      builder.addAction(MultiAction.newBuilder().setMutation(mp).build());
     }
-    return createMultiRequest(builder.build());
+    return builder.build();
   }
 
   /**
@@ -399,8 +398,7 @@ public final class RequestConverter {
   public static MultiRequest buildNoDataMultiRequest(final byte[] regionName,
       final RowMutations rowMutations, final List<CellScannable> cells)
   throws IOException {
-    RegionMutation.Builder builder =
-      getRegionMutationBuilderWithRegionAndAtomicSet(regionName, true);
+    MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName,
true);
     for (Mutation mutation: rowMutations.getMutations()) {
       MutationType type = null;
       if (mutation instanceof Put) {
@@ -413,18 +411,14 @@ public final class RequestConverter {
       }
       MutationProto mp = ProtobufUtil.toMutationNoData(type, mutation);
       cells.add(mutation);
-      builder.addMutation(mp);
+      builder.addAction(MultiAction.newBuilder().setMutation(mp).build());
     }
-    return createMultiRequest(builder.build());
-  }
-
-  private static MultiRequest createMultiRequest(final RegionMutation rm) {
-    return MultiRequest.newBuilder().addRegionMutation(rm).build();
+    return builder.build();
   }
 
-  private static RegionMutation.Builder getRegionMutationBuilderWithRegionAndAtomicSet(
-      final byte [] regionName, final boolean atomic) {
-    RegionMutation.Builder builder = RegionMutation.newBuilder();
+  private static MultiRequest.Builder getMultiRequestBuilderWithRegionAndAtomicSet(final
byte [] regionName,
+      final boolean atomic) {
+    MultiRequest.Builder builder = MultiRequest.newBuilder();
     RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
     builder.setRegion(region);
     return builder.setAtomic(atomic);
@@ -526,27 +520,29 @@ public final class RequestConverter {
   public static <R> MultiRequest buildMultiRequest(final byte[] regionName,
       final List<Action<R>> actions)
   throws IOException {
-    RegionMutation.Builder builder =
-      getRegionMutationBuilderWithRegionAndAtomicSet(regionName, false);
+    MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName,
false);
     for (Action<R> action: actions) {
+      MultiAction.Builder protoAction = MultiAction.newBuilder();
       Row row = action.getAction();
       if (row instanceof Get) {
-        throw new UnsupportedOperationException("Removed");
+        protoAction.setGet(ProtobufUtil.toGet((Get)row));
       } else if (row instanceof Put) {
-        builder.addMutation(ProtobufUtil.toMutation(MutationType.PUT, (Put)row));
+        protoAction.setMutation(ProtobufUtil.toMutation(MutationType.PUT, (Put)row));
       } else if (row instanceof Delete) {
-        builder.addMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row));
+        protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row));
       } else if (row instanceof Append) {
-        builder.addMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row));
+        protoAction.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row));
       } else if (row instanceof Increment) {
-        builder.addMutation(ProtobufUtil.toMutation((Increment)row));
+        protoAction.setMutation(ProtobufUtil.toMutation((Increment)row));
       } else if (row instanceof RowMutations) {
         continue; // ignore RowMutations
       } else {
-        throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
+        throw new DoNotRetryIOException(
+          "multi doesn't support " + row.getClass().getName());
       }
+      builder.addAction(protoAction.build());
     }
-    return createMultiRequest(builder.build());
+    return builder.build();
   }
 
   /**
@@ -568,16 +564,17 @@ public final class RequestConverter {
   public static <R> MultiRequest buildNoDataMultiRequest(final byte[] regionName,
       final List<Action<R>> actions, final List<CellScannable> cells)
   throws IOException {
-    RegionMutation.Builder builder =
-        getRegionMutationBuilderWithRegionAndAtomicSet(regionName, false);
+    MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName,
false);
     for (Action<R> action: actions) {
+      MultiAction.Builder protoAction = MultiAction.newBuilder();
       Row row = action.getAction();
       if (row instanceof Get) {
-        throw new UnsupportedOperationException("Removed");
+        // Gets are carried by protobufs.
+        protoAction.setGet(ProtobufUtil.toGet((Get)row));
       } else if (row instanceof Put) {
         Put p = (Put)row;
         cells.add(p);
-        builder.addMutation(ProtobufUtil.toMutationNoData(MutationType.PUT, p));
+        protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.PUT, p));
       } else if (row instanceof Delete) {
         Delete d = (Delete)row;
         int size = d.size();
@@ -588,25 +585,26 @@ public final class RequestConverter {
         // metadata only in the pb and then send the kv along the side in cells.
         if (size > 0) {
           cells.add(d);
-          builder.addMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d));
+          protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d));
         } else {
-          builder.addMutation(ProtobufUtil.toMutation(MutationType.DELETE, d));
+          protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, d));
         }
       } else if (row instanceof Append) {
         Append a = (Append)row;
         cells.add(a);
-        builder.addMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a));
+        protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a));
       } else if (row instanceof Increment) {
         Increment i = (Increment)row;
         cells.add(i);
-        builder.addMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i));
+        protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i));
       } else if (row instanceof RowMutations) {
         continue; // ignore RowMutations
       } else {
         throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
       }
+      builder.addAction(protoAction.build());
     }
-    return createMultiRequest(builder.build());
+    return builder.build();
   }
 
 // End utilities for Client

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
(original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
Mon Sep 30 21:27:33 2013
@@ -38,10 +38,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionMutationResult;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
@@ -74,20 +73,20 @@ public final class ResponseConverter {
    * @return the results that were in the MultiResponse (a Result or an Exception).
    * @throws IOException
    */
-  public static List<Object> getResults(final MultiResponse proto,
+  public static List<Object> getResults(final ClientProtos.MultiResponse proto,
       final CellScanner cells)
   throws IOException {
     List<Object> results = new ArrayList<Object>();
-    for (RegionMutationResult result: proto.getRegionMutationResultList()) {
-      for (ResultOrException resultOrException: result.getResultOrExceptionList()) {
-        if (resultOrException.hasException()) {
-          results.add(ProtobufUtil.toException(resultOrException.getException()));
-        } else if (resultOrException.hasResult()) {
-          results.add(ProtobufUtil.toResult(resultOrException.getResult(), cells));
-        } else {
-          // Just a placeholder
-          results.add(new Result());
-        }
+    List<ActionResult> resultList = proto.getResultList();
+    for (int i = 0, n = resultList.size(); i < n; i++) {
+      ActionResult result = resultList.get(i);
+      if (result.hasException()) {
+        results.add(ProtobufUtil.toException(result.getException()));
+      } else if (result.hasValue()) {
+        ClientProtos.Result value = result.getValue();
+        results.add(ProtobufUtil.toResult(value, cells));
+      } else {
+        results.add(new Result());
       }
     }
     return results;
@@ -99,22 +98,14 @@ public final class ResponseConverter {
    * @param t
    * @return an action result
    */
-  public static ResultOrException buildActionResult(final Throwable t) {
-    ResultOrException.Builder builder = ResultOrException.newBuilder();
-    builder.setException(buildException(t));
-    return builder.build();
-  }
-
-  /**
-   * @param t
-   * @return NameValuePair of the exception name to stringified version os exception.
-   */
-  public static NameBytesPair buildException(final Throwable t) {
+  public static ActionResult buildActionResult(final Throwable t) {
+    ActionResult.Builder builder = ActionResult.newBuilder();
     NameBytesPair.Builder parameterBuilder = NameBytesPair.newBuilder();
     parameterBuilder.setName(t.getClass().getName());
     parameterBuilder.setValue(
       ByteString.copyFromUtf8(StringUtils.stringifyException(t)));
-    return parameterBuilder.build();
+    builder.setException(parameterBuilder.build());
+    return builder.build();
   }
 
   /**

Modified: hbase/trunk/hbase-protocol/README.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/README.txt?rev=1527789&r1=1527788&r2=1527789&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/README.txt (original)
+++ hbase/trunk/hbase-protocol/README.txt Mon Sep 30 21:27:33 2013
@@ -25,6 +25,7 @@ terminal and hit return -- the protoc co
   do
     protoc -I$PROTO_DIR --java_out=$JAVA_DIR $PROTO_FILE
   done
+  ll $JAVA_DIR/org/apache/hadoop/hbase/protobuf/generated
 
 After you've done the above, check it in and then check it in (or post a patch
 on a JIRA with your definition file changes and the generated files).



Mime
View raw message