diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 4e43ac4..6e6b155 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -255,12 +255,14 @@ class AsyncProcess { * @param atLeastOne true if we should submit at least a subset. */ public void submit(List rows, boolean atLeastOne) throws InterruptedIOException { - if (rows.isEmpty()){ + if (rows.isEmpty()) { return; } + // This looks like we are keying by region but HRegionLocation has a comparator that compares + // on the server portion only (hostname + port) so this Map collects regions by server. Map> actionsByServer = - new HashMap>(); + new HashMap>(); List> retainedActions = new ArrayList>(rows.size()); do { @@ -321,10 +323,7 @@ class AsyncProcess { * @return the destination. Null if we couldn't find it. */ private HRegionLocation findDestLocation(Row row, int numAttempt, int posInList) { - if (row == null){ - throw new IllegalArgumentException("row cannot be null"); - } - + if (row == null) throw new IllegalArgumentException("row cannot be null"); HRegionLocation loc = null; IOException locationException = null; try { @@ -476,29 +475,29 @@ class AsyncProcess { final int numAttempt, final HConnectionManager.ServerErrorTracker errorsByServer) { // Send the queries and add them to the inProgress list + // This iteration is by server (the HRegionLocation comparator is by server portion only). for (Map.Entry> e : actionsByServer.entrySet()) { final HRegionLocation loc = e.getKey(); - final MultiAction multi = e.getValue(); - incTaskCounters(multi.getRegions(), loc.getServerName()); - + final MultiAction multiAction = e.getValue(); + incTaskCounters(multiAction.getRegions(), loc.getServerName()); Runnable runnable = Trace.wrap("AsyncProcess.sendMultiAction", new Runnable() { @Override public void run() { MultiResponse res; try { - MultiServerCallable callable = createCallable(loc, multi); + MultiServerCallable callable = createCallable(loc, multiAction); try { res = createCaller(callable).callWithoutRetries(callable); } catch (IOException e) { - LOG.warn("The call to the region server failed, we don't know where we stand, " + - loc.getServerName(), e); - resubmitAll(initialActions, multi, loc, numAttempt + 1, e, errorsByServer); + LOG.warn("Call to " + loc.getServerName() + " failed numAttempt=" + numAttempt + + ", resubmitting all since not sure where we are at", e); + resubmitAll(initialActions, multiAction, loc, numAttempt + 1, e, errorsByServer); return; } - receiveMultiAction(initialActions, multi, loc, res, numAttempt, errorsByServer); + receiveMultiAction(initialActions, multiAction, loc, res, numAttempt, errorsByServer); } finally { - decTaskCounters(multi.getRegions(), loc.getServerName()); + decTaskCounters(multiAction.getRegions(), loc.getServerName()); } } }); @@ -508,12 +507,12 @@ class AsyncProcess { } catch (RejectedExecutionException ree) { // This should never happen. But as the pool is provided by the end user, let's secure // this a little. - decTaskCounters(multi.getRegions(), loc.getServerName()); + decTaskCounters(multiAction.getRegions(), loc.getServerName()); LOG.warn("The task was rejected by the pool. This is unexpected." + " Server is " + loc.getServerName(), ree); // We're likely to fail again, but this will increment the attempt counter, so it will // finish. - resubmitAll(initialActions, multi, loc, numAttempt + 1, ree, errorsByServer); + resubmitAll(initialActions, multiAction, loc, numAttempt + 1, ree, errorsByServer); } } } @@ -590,10 +589,9 @@ class AsyncProcess { // Do not use the exception for updating cache because it might be coming from // any of the regions in the MultiAction. hConnection.updateCachedLocations(tableName, - rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location); + rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location); errorsByServer.reportServerError(location); - - List> toReplay = new ArrayList>(); + List> toReplay = new ArrayList>(initialActions.size()); for (List> actions : rsActions.actions.values()) { for (Action action : actions) { if (manageError(numAttempt, action.getOriginalIndex(), action.getAction(), @@ -605,7 +603,7 @@ class AsyncProcess { if (toReplay.isEmpty()) { LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all " + - initialActions.size() + "ops, NOT resubmitting, " + location.getServerName()); + initialActions.size() + " ops, NOT resubmitting, " + location.getServerName()); } else { submit(initialActions, toReplay, numAttempt, errorsByServer); } @@ -669,11 +667,11 @@ class AsyncProcess { } } else { // success if (callback != null) { - Action correspondingAction = initialActions.get(regionResult.getFirst()); + int index = regionResult.getFirst(); + Action correspondingAction = initialActions.get(index); Row row = correspondingAction.getAction(); //noinspection unchecked - this.callback.success(correspondingAction.getOriginalIndex(), - resultsForRS.getKey(), row, (CResult) result); + this.callback.success(index, resultsForRS.getKey(), row, (CResult) result); } } } @@ -694,8 +692,7 @@ class AsyncProcess { try { Thread.sleep(backOffTime); } catch (InterruptedException e) { - LOG.warn("Not sent: " + toReplay.size() + - " operations, " + location, e); + LOG.warn("Not sent: " + toReplay.size() + " operations, " + location, e); Thread.interrupted(); return; } @@ -708,7 +705,8 @@ class AsyncProcess { " ops on " + location.getServerName() + " NOT resubmitting." + location); } else if (numAttempt > START_LOG_ERRORS_CNT + 1 && LOG.isDebugEnabled()) { // The operation was successful, but needed several attempts. Let's log this. - LOG.debug("Attempt #" + numAttempt + "/" + numTries + " is finally successful."); + LOG.debug("Attempt #" + numAttempt + "/" + numTries + " finally suceeded, size=" + + toReplay.size()); } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java index 0b89544..a17be55 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java @@ -171,6 +171,7 @@ public class ClientSmallScanner extends ClientScanner { ScanResponse response = null; PayloadCarryingRpcController controller = new PayloadCarryingRpcController(); try { + controller.setPriority(getTableName()); response = getStub().scan(controller, request); return ResponseConverter.getResults(controller.cellScanner(), response); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java index cc07d87..2c45a6d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java @@ -32,7 +32,6 @@ import java.util.TreeSet; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 32fe4f0..017f311 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -643,6 +643,7 @@ public class HBaseAdmin implements Abortable, Closeable { .getServerName()); PayloadCarryingRpcController controller = new PayloadCarryingRpcController(); try { + controller.setPriority(tableName); ScanResponse response = server.scan(controller, request); values = ResponseConverter.getResults(controller.cellScanner(), response); } catch (ServiceException se) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 89efe2d..50fb36c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -2165,8 +2165,7 @@ public class HConnectionManager { boolean isStaleDelete = false; HRegionLocation oldLocation; synchronized (this.cachedRegionLocations) { - Map tableLocations = - getTableLocations(hri.getTable()); + Map tableLocations = getTableLocations(hri.getTable()); oldLocation = tableLocations.get(hri.getStartKey()); if (oldLocation != null) { // Do not delete the cache entry if it's not for the same server that gave us the error. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index c49d41b..a825b23 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -959,8 +960,13 @@ public class HTable implements HTableInterface { new RegionServerCallable(connection, getName(), rm.getRow()) { public Void call() throws IOException { try { - MultiRequest request = RequestConverter.buildMultiRequest( + RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction( getLocation().getRegionInfo().getRegionName(), rm); + regionMutationBuilder.setAtomic(true); + MultiRequest request = + MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build(); + PayloadCarryingRpcController pcrc = new PayloadCarryingRpcController(); + pcrc.setPriority(tableName); getStub().multi(null, request); } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); @@ -987,6 +993,7 @@ public class HTable implements HTableInterface { 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()); @@ -1013,9 +1020,10 @@ public class HTable implements HTableInterface { try { MutateRequest request = RequestConverter.buildMutateRequest( getLocation().getRegionInfo().getRegionName(), increment); - PayloadCarryingRpcController rpcContoller = new PayloadCarryingRpcController(); - MutateResponse response = getStub().mutate(rpcContoller, request); - return ProtobufUtil.toResult(response.getResult(), rpcContoller.cellScanner()); + PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController(); + rpcController.setPriority(getTableName()); + MutateResponse response = getStub().mutate(rpcController, request); + return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner()); } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); } @@ -1074,6 +1082,7 @@ public class HTable implements HTableInterface { 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()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java index 2ce6118..c0e653d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java @@ -24,14 +24,17 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScannable; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction; +import org.apache.hadoop.hbase.util.Pair; import com.google.protobuf.ServiceException; @@ -42,91 +45,74 @@ import com.google.protobuf.ServiceException; * @param */ class MultiServerCallable extends RegionServerCallable { - private final MultiAction multi; + private final MultiAction multiAction; private final boolean cellBlock; MultiServerCallable(final HConnection connection, final TableName tableName, final HRegionLocation location, final MultiAction multi) { super(connection, tableName, null); - this.multi = multi; + this.multiAction = multi; setLocation(location); this.cellBlock = isCellBlock(); } MultiAction getMulti() { - return this.multi; + return this.multiAction; } @Override public MultiResponse call() throws IOException { - MultiResponse response = new MultiResponse(); - // The multi object is a list of Actions by region. - for (Map.Entry>> e: this.multi.actions.entrySet()) { - byte[] regionName = e.getKey(); - int rowMutations = 0; - List> actions = e.getValue(); - for (Action action : actions) { - Row row = action.getAction(); - // Row Mutations are a set of Puts and/or Deletes all to be applied atomically - // on the one row. We do these a row at a time. - if (row instanceof RowMutations) { - RowMutations rms = (RowMutations)row; - List cells = null; - MultiRequest multiRequest; - try { - if (this.cellBlock) { - // Stick all Cells for all RowMutations in here into 'cells'. Populated when we call - // buildNoDataMultiRequest in the below. - cells = new ArrayList(rms.getMutations().size()); - // Build a multi request absent its Cell payload (this is the 'nodata' in the below). - multiRequest = RequestConverter.buildNoDataMultiRequest(regionName, rms, cells); - } else { - multiRequest = RequestConverter.buildMultiRequest(regionName, rms); - } - // Carry the cells if any over the proxy/pb Service interface using the payload - // 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) { - response.add(regionName, action.getOriginalIndex(), - ProtobufUtil.getRemoteException(se)); - } - rowMutations++; - } - } - // Are there any non-RowMutation actions to send for this region? - if (actions.size() > rowMutations) { - Exception ex = null; - List results = null; - List cells = null; - MultiRequest multiRequest; - try { - if (isCellBlock()) { - // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations. - // They have already been handled above. - cells = new ArrayList(actions.size() - rowMutations); - multiRequest = RequestConverter.buildNoDataMultiRequest(regionName, actions, cells); - } else { - multiRequest = RequestConverter.buildMultiRequest(regionName, actions); - } - // 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); - ClientProtos.MultiResponse responseProto = getStub().multi(controller, multiRequest); - results = ResponseConverter.getResults(responseProto, controller.cellScanner()); - } catch (ServiceException se) { - ex = ProtobufUtil.getRemoteException(se); - } - for (int i = 0, n = actions.size(); i < n; i++) { - int originalIndex = actions.get(i).getOriginalIndex(); - response.add(regionName, originalIndex, results == null ? ex : results.get(i)); - } + int countOfActions = this.multiAction.size(); + if (countOfActions <= 0) throw new HBaseIOException("No Actions"); + MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder(); + List cells = null; + // The multi object is a list of Actions by region. Iterate by region. + for (Map.Entry>> e: this.multiAction.actions.entrySet()) { + final byte [] regionName = e.getKey(); + final List> actions = e.getValue(); + RegionAction.Builder regionActionBuilder = null; + if (this.cellBlock) { + // Presize. Presume at least a KV per Action. There are likely more. + if (cells == null) cells = new ArrayList(countOfActions); + // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations. + // They have already been handled above. Guess at count of cells + regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells); + } else { + regionActionBuilder = RequestConverter.buildRegionAction(regionName, actions); } + multiRequestBuilder.addRegionAction(regionActionBuilder.build()); } - return response; + // 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; + ClientProtos.MultiRequest requestProto = multiRequestBuilder.build(); + try { + responseProto = getStub().multi(controller, requestProto); + } catch (ServiceException e) { + return createAllFailedResponse(requestProto, ProtobufUtil.getRemoteException(e)); + } + return ResponseConverter.getResults(requestProto, responseProto, controller.cellScanner()); } + /** + * @param request + * @param t + * @return Return a response that has every action in request failed w/ the passed in + * exception t -- this will get them all retried after some backoff. + */ + private static MultiResponse createAllFailedResponse(final ClientProtos.MultiRequest request, + final Throwable t) { + MultiResponse massFailedResponse = new MultiResponse(); + for (RegionAction rAction: request.getRegionActionList()) { + byte [] regionName = rAction.getRegion().getValue().toByteArray(); + for (ClientProtos.Action action: rAction.getActionList()) { + massFailedResponse.add(regionName, new Pair(action.getIndex(), t)); + } + } + return massFailedResponse; + } /** * @return True if we should send data in cellblocks. This is an expensive call. Cache the diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java index 5df1771..3f25fcb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java @@ -163,6 +163,7 @@ public class ScannerCallable extends RegionServerCallable { 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 diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java index d3bc831..1f9b4bb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java @@ -23,7 +23,8 @@ import org.apache.hadoop.classification.InterfaceAudience; 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; @@ -36,6 +37,15 @@ 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 /** @@ -103,4 +113,26 @@ public class PayloadCarryingRpcController implements RpcController, CellScannabl 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 diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java index 5d607a6..c2f782b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java @@ -55,6 +55,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.codec.Codec; @@ -211,7 +212,8 @@ public class RpcClient { @SuppressWarnings("serial") @InterfaceAudience.Public @InterfaceStability.Evolving - public static class FailedServerException extends IOException { + // Shouldn't this be a DoNotRetryException? St.Ack 10/2/2013 + public static class FailedServerException extends HBaseIOException { public FailedServerException(String s) { super(s); } @@ -967,8 +969,12 @@ public class RpcClient { } // close the streams and therefore the socket - IOUtils.closeStream(out); - this.out = null; + if (this.out != null) { + synchronized(this.out) { + IOUtils.closeStream(out); + this.out = null; + } + } IOUtils.closeStream(in); this.in = null; disposeSasl(); @@ -1002,9 +1008,10 @@ 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) { + protected void writeRequest(Call call, final int priority) { if (shouldCloseConnection.get()) return; try { RequestHeader.Builder builder = RequestHeader.newBuilder(); @@ -1022,6 +1029,8 @@ 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 @@ -1380,6 +1389,12 @@ public class RpcClient { } } + Pair 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 param, to the IPC server running at * address which is servicing the protocol protocol, * with the ticket credentials, returning the value. @@ -1400,12 +1415,12 @@ public class RpcClient { */ Pair call(MethodDescriptor md, Message param, CellScanner cells, Message returnType, User ticket, InetSocketAddress addr, - int rpcTimeout) + int rpcTimeout, int priority) 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); // send the parameter + connection.writeRequest(call, priority); // send the parameter boolean interrupted = false; //noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (call) { @@ -1632,7 +1647,8 @@ public class RpcClient { } Pair val = null; try { - val = call(md, param, cells, returnType, ticket, isa, rpcTimeout); + val = call(md, param, cells, returnType, ticket, isa, rpcTimeout, + pcrc != null? pcrc.getPriority(): HConstants.NORMAL_QOS); 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()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 24b328d..9d17b0a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -126,6 +126,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.security.token.Token; +import org.mortbay.log.Log; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; @@ -2257,11 +2258,15 @@ public final class ProtobufUtil { ", row=" + getStringForByteString(r.getGet().getRow()); } else if (m instanceof ClientProtos.MultiRequest) { ClientProtos.MultiRequest r = (ClientProtos.MultiRequest) m; - 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()); + // Get first set of Actions. + ClientProtos.RegionAction actions = r.getRegionActionList().get(0); + String row = actions.getActionCount() <= 0? "": + getStringForByteString(actions.getAction(0).hasGet()? + actions.getAction(0).getGet().getRow(): + actions.getAction(0).getMutation().getRow()); + return "region= " + getStringForByteString(actions.getRegion().getValue()) + + ", for " + r.getRegionActionCount() + + " actions and 1st row key=" + row; } else if (m instanceof ClientProtos.MutateRequest) { ClientProtos.MutateRequest r = (ClientProtos.MutateRequest) m; return "region= " + getStringForByteString(r.getRegion().getValue()) + diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index bf37d3a..b7bffde 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -63,14 +63,13 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ 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; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; 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.RegionAction; 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; @@ -101,6 +100,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLa import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Triple; +import org.mortbay.log.Log; import com.google.protobuf.ByteString; @@ -358,17 +358,18 @@ public final class RequestConverter { } /** - * Create a protocol buffer MultiRequest for a row mutations - * + * Create a protocol buffer MultiRequest for row mutations. + * Does not propagate Action absolute position. Does not set atomic action on the created + * RegionAtomic. Caller should do that if wanted. * @param regionName * @param rowMutations - * @return a multi request + * @return a data-laden RegionMutation.Builder * @throws IOException */ - public static MultiRequest buildMultiRequest(final byte[] regionName, + public static RegionAction.Builder buildRegionAction(final byte [] regionName, final RowMutations rowMutations) throws IOException { - MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, true); + RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName); for (Mutation mutation: rowMutations.getMutations()) { MutationType mutateType = null; if (mutation instanceof Put) { @@ -380,25 +381,26 @@ public final class RequestConverter { mutation.getClass().getName()); } MutationProto mp = ProtobufUtil.toMutation(mutateType, mutation); - builder.addAction(MultiAction.newBuilder().setMutation(mp).build()); + builder.addAction(ClientProtos.Action.newBuilder().setMutation(mp).build()); } - return builder.build(); + return builder; } /** * Create a protocol buffer MultiRequest for row mutations that does not hold data. Data/Cells - * are carried outside of protobuf. Return references to the Cells in cells param - * + * are carried outside of protobuf. Return references to the Cells in cells param. + * Does not propagate Action absolute position. Does not set atomic action on the created + * RegionAtomic. Caller should do that if wanted. * @param regionName * @param rowMutations * @param cells Return in here a list of Cells as CellIterable. - * @return a multi request minus data + * @return a region mutation minus data * @throws IOException */ - public static MultiRequest buildNoDataMultiRequest(final byte[] regionName, + public static RegionAction.Builder buildNoDataRegionAction(final byte[] regionName, final RowMutations rowMutations, final List cells) throws IOException { - MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, true); + RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName); for (Mutation mutation: rowMutations.getMutations()) { MutationType type = null; if (mutation instanceof Put) { @@ -411,17 +413,16 @@ public final class RequestConverter { } MutationProto mp = ProtobufUtil.toMutationNoData(type, mutation); cells.add(mutation); - builder.addAction(MultiAction.newBuilder().setMutation(mp).build()); + builder.addAction(ClientProtos.Action.newBuilder().setMutation(mp).build()); } - return builder.build(); + return builder; } - private static MultiRequest.Builder getMultiRequestBuilderWithRegionAndAtomicSet(final byte [] regionName, - final boolean atomic) { - MultiRequest.Builder builder = MultiRequest.newBuilder(); + private static RegionAction.Builder getRegionActionBuilderWithRegion(final byte [] regionName) { + RegionAction.Builder builder = RegionAction.newBuilder(); RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName); builder.setRegion(region); - return builder.setAtomic(atomic); + return builder; } /** @@ -510,39 +511,43 @@ public final class RequestConverter { /** * Create a protocol buffer multi request for a list of actions. - * RowMutations in the list (if any) will be ignored. + * Propagates Actions original index. * * @param regionName * @param actions * @return a multi request * @throws IOException */ - public static MultiRequest buildMultiRequest(final byte[] regionName, + public static RegionAction.Builder buildRegionAction(final byte[] regionName, final List> actions) throws IOException { - MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false); + RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName); for (Action action: actions) { - MultiAction.Builder protoAction = MultiAction.newBuilder(); Row row = action.getAction(); + ClientProtos.Action.Builder actionBuilder = + ClientProtos.Action.newBuilder().setIndex(action.getOriginalIndex()); if (row instanceof Get) { - protoAction.setGet(ProtobufUtil.toGet((Get)row)); + Get g = (Get)row; + builder.addAction(actionBuilder.setGet(ProtobufUtil.toGet(g))); } else if (row instanceof Put) { - protoAction.setMutation(ProtobufUtil.toMutation(MutationType.PUT, (Put)row)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutation(MutationType.PUT, (Put)row))); } else if (row instanceof Delete) { - protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row))); } else if (row instanceof Append) { - protoAction.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row))); } else if (row instanceof Increment) { - protoAction.setMutation(ProtobufUtil.toMutation((Increment)row)); + builder.addAction(actionBuilder. + 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 builder.build(); + return builder; } /** @@ -553,7 +558,7 @@ public final class RequestConverter { * carried by protobuf. We return references to the data by adding them to the passed in * data param. * - * RowMutations in the list (if any) will be ignored. + *

Propagates Actions original index. * * @param regionName * @param actions @@ -561,20 +566,22 @@ public final class RequestConverter { * @return a multi request that does not carry any data. * @throws IOException */ - public static MultiRequest buildNoDataMultiRequest(final byte[] regionName, + public static RegionAction.Builder buildNoDataRegionAction(final byte[] regionName, final List> actions, final List cells) throws IOException { - MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false); + RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName); for (Action action: actions) { - MultiAction.Builder protoAction = MultiAction.newBuilder(); Row row = action.getAction(); + ClientProtos.Action.Builder actionBuilder = + ClientProtos.Action.newBuilder().setIndex(action.getOriginalIndex()); if (row instanceof Get) { - // Gets are carried by protobufs. - protoAction.setGet(ProtobufUtil.toGet((Get)row)); + Get g = (Get)row; + builder.addAction(actionBuilder.setGet(ProtobufUtil.toGet(g))); } else if (row instanceof Put) { Put p = (Put)row; cells.add(p); - protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.PUT, p)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutationNoData(MutationType.PUT, p))); } else if (row instanceof Delete) { Delete d = (Delete)row; int size = d.size(); @@ -585,26 +592,29 @@ 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); - protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d))); } else { - protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, d)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutation(MutationType.DELETE, d))); } } else if (row instanceof Append) { Append a = (Append)row; cells.add(a); - protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a)); + builder.addAction(actionBuilder. + setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a))); } else if (row instanceof Increment) { Increment i = (Increment)row; cells.add(i); - protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i)); + builder.addAction(actionBuilder. + 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 builder.build(); + return builder; } // End utilities for Client diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java index 50a4838..3260ef6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java @@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Result; @@ -39,14 +40,19 @@ 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; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException; 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; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; import org.apache.hadoop.hbase.security.access.UserPermission; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.util.StringUtils; import com.google.protobuf.ByteString; @@ -73,20 +79,34 @@ public final class ResponseConverter { * @return the results that were in the MultiResponse (a Result or an Exception). * @throws IOException */ - public static List getResults(final ClientProtos.MultiResponse proto, - final CellScanner cells) + public static org.apache.hadoop.hbase.client.MultiResponse getResults(final MultiRequest request, + final MultiResponse response, final CellScanner cells) throws IOException { - List results = new ArrayList(); - List 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()); + int requestRegionActionCount = request.getRegionActionCount(); + int responseRegionActionResultCount = response.getRegionActionResultCount(); + if (requestRegionActionCount != responseRegionActionResultCount) { + throw new HBaseIOException("Request mutation count=" + responseRegionActionResultCount + + " does not match response mutation result count=" + responseRegionActionResultCount); + } + org.apache.hadoop.hbase.client.MultiResponse results = + new org.apache.hadoop.hbase.client.MultiResponse(); + for (int i = 0; i < responseRegionActionResultCount; i++) { + RegionAction actions = request.getRegionAction(i); + RegionActionResult actionResult = response.getRegionActionResult(i); + byte [] regionName = actions.getRegion().toByteArray(); + int count = actionResult.getResultOrExceptionCount(); + for (int ii = 0; ii < count; ii++) { + ResultOrException roe = actionResult.getResultOrException(ii); + if (roe.hasException()) { + results.add(regionName, new Pair(roe.getIndex(), + ProtobufUtil.toException(roe.getException()))); + } else if (roe.hasResult()) { + results.add(regionName, new Pair(roe.getIndex(), + ProtobufUtil.toResult(roe.getResult(), cells))); + } else { + // Just a placeholder + results.add(regionName, new Pair(roe.getIndex(), new Result())); + } } } return results; @@ -96,16 +116,36 @@ public final class ResponseConverter { * Wrap a throwable to an action result. * * @param t - * @return an action result + * @return an action result builder + */ + public static ResultOrException.Builder buildActionResult(final Throwable t) { + ResultOrException.Builder builder = ResultOrException.newBuilder(); + if (t != null) builder.setException(buildException(t)); + return builder; + } + + /** + * Wrap a throwable to an action result. + * + * @param t + * @return an action result builder + */ + public static ResultOrException.Builder buildActionResult(final ClientProtos.Result r) { + ResultOrException.Builder builder = ResultOrException.newBuilder(); + if (r != null) builder.setResult(r); + return builder; + } + + /** + * @param t + * @return NameValuePair of the exception name to stringified version os exception. */ - public static ActionResult buildActionResult(final Throwable t) { - ActionResult.Builder builder = ActionResult.newBuilder(); + public static NameBytesPair buildException(final Throwable t) { NameBytesPair.Builder parameterBuilder = NameBytesPair.newBuilder(); parameterBuilder.setName(t.getClass().getName()); parameterBuilder.setValue( ByteString.copyFromUtf8(StringUtils.stringifyException(t))); - builder.setException(parameterBuilder.build()); - return builder.build(); + return parameterBuilder.build(); } /** diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 2940572..d19e391 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -301,4 +301,4 @@ public class TestClientNoCluster { return this.stub; } } -} +} \ No newline at end of file diff --git a/hbase-protocol/README.txt b/hbase-protocol/README.txt index 24e2b75..b8e21ae 100644 --- a/hbase-protocol/README.txt +++ b/hbase-protocol/README.txt @@ -25,7 +25,6 @@ terminal and hit return -- the protoc compiler runs fast): 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). diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index 838c35b..ebfd08a 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -23667,62 +23667,80 @@ public final class ClientProtos { // @@protoc_insertion_point(class_scope:CoprocessorServiceResponse) } - public interface MultiActionOrBuilder + public interface ActionOrBuilder extends com.google.protobuf.MessageOrBuilder { - // optional .MutationProto mutation = 1; + // optional uint32 index = 1; /** - * optional .MutationProto mutation = 1; + * optional uint32 index = 1; + * + *
+     * If part of a multi action, useful aligning
+     * result with what was originally submitted.
+     * 
+ */ + boolean hasIndex(); + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi action, useful aligning
+     * result with what was originally submitted.
+     * 
+ */ + int getIndex(); + + // optional .MutationProto mutation = 2; + /** + * optional .MutationProto mutation = 2; */ boolean hasMutation(); /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation(); /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder(); - // optional .Get get = 2; + // optional .Get get = 3; /** - * optional .Get get = 2; + * optional .Get get = 3; */ boolean hasGet(); /** - * optional .Get get = 2; + * optional .Get get = 3; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get getGet(); /** - * optional .Get get = 2; + * optional .Get get = 3; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder(); } /** - * Protobuf type {@code MultiAction} + * Protobuf type {@code Action} * *
-   **
-   * An action that is part of MultiRequest.
-   * This is a union type - exactly one of the fields will be set.
+   * Either a Get or a Mutation
    * 
*/ - public static final class MultiAction extends + public static final class Action extends com.google.protobuf.GeneratedMessage - implements MultiActionOrBuilder { - // Use MultiAction.newBuilder() to construct. - private MultiAction(com.google.protobuf.GeneratedMessage.Builder builder) { + implements ActionOrBuilder { + // Use Action.newBuilder() to construct. + private Action(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); this.unknownFields = builder.getUnknownFields(); } - private MultiAction(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private Action(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - private static final MultiAction defaultInstance; - public static MultiAction getDefaultInstance() { + private static final Action defaultInstance; + public static Action getDefaultInstance() { return defaultInstance; } - public MultiAction getDefaultInstanceForType() { + public Action getDefaultInstanceForType() { return defaultInstance; } @@ -23732,7 +23750,7 @@ public final class ClientProtos { getUnknownFields() { return this.unknownFields; } - private MultiAction( + private Action( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -23755,9 +23773,14 @@ public final class ClientProtos { } break; } - case 10: { + case 8: { + bitField0_ |= 0x00000001; + index_ = input.readUInt32(); + break; + } + case 18: { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) == 0x00000001)) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { subBuilder = mutation_.toBuilder(); } mutation_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.PARSER, extensionRegistry); @@ -23765,12 +23788,12 @@ public final class ClientProtos { subBuilder.mergeFrom(mutation_); mutation_ = subBuilder.buildPartial(); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; break; } - case 18: { + case 26: { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) == 0x00000002)) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { subBuilder = get_.toBuilder(); } get_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.PARSER, extensionRegistry); @@ -23778,7 +23801,7 @@ public final class ClientProtos { subBuilder.mergeFrom(get_); get_ = subBuilder.buildPartial(); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; break; } } @@ -23795,77 +23818,104 @@ public final class ClientProtos { } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiAction_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Action_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiAction_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Action_fieldAccessorTable .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder.class); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder.class); } - public static com.google.protobuf.Parser PARSER = - new com.google.protobuf.AbstractParser() { - public MultiAction parsePartialFrom( + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public Action parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new MultiAction(input, extensionRegistry); + return new Action(input, extensionRegistry); } }; @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public com.google.protobuf.Parser getParserForType() { return PARSER; } private int bitField0_; - // optional .MutationProto mutation = 1; - public static final int MUTATION_FIELD_NUMBER = 1; + // optional uint32 index = 1; + public static final int INDEX_FIELD_NUMBER = 1; + private int index_; + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi action, useful aligning
+     * result with what was originally submitted.
+     * 
+ */ + public boolean hasIndex() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi action, useful aligning
+     * result with what was originally submitted.
+     * 
+ */ + public int getIndex() { + return index_; + } + + // optional .MutationProto mutation = 2; + public static final int MUTATION_FIELD_NUMBER = 2; private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto mutation_; /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public boolean hasMutation() { - return ((bitField0_ & 0x00000001) == 0x00000001); + return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation() { return mutation_; } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder() { return mutation_; } - // optional .Get get = 2; - public static final int GET_FIELD_NUMBER = 2; + // optional .Get get = 3; + public static final int GET_FIELD_NUMBER = 3; private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get get_; /** - * optional .Get get = 2; + * optional .Get get = 3; */ public boolean hasGet() { - return ((bitField0_ & 0x00000002) == 0x00000002); + return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get getGet() { return get_; } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder() { return get_; } private void initFields() { + index_ = 0; mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance(); } @@ -23894,10 +23944,13 @@ public final class ClientProtos { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, mutation_); + output.writeUInt32(1, index_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, get_); + output.writeMessage(2, mutation_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, get_); } getUnknownFields().writeTo(output); } @@ -23910,11 +23963,15 @@ public final class ClientProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, mutation_); + .computeUInt32Size(1, index_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, get_); + .computeMessageSize(2, mutation_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, get_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -23933,12 +23990,17 @@ public final class ClientProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction) obj; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action) obj; boolean result = true; + result = result && (hasIndex() == other.hasIndex()); + if (hasIndex()) { + result = result && (getIndex() + == other.getIndex()); + } result = result && (hasMutation() == other.hasMutation()); if (hasMutation()) { result = result && getMutation() @@ -23962,6 +24024,10 @@ public final class ClientProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasIndex()) { + hash = (37 * hash) + INDEX_FIELD_NUMBER; + hash = (53 * hash) + getIndex(); + } if (hasMutation()) { hash = (37 * hash) + MUTATION_FIELD_NUMBER; hash = (53 * hash) + getMutation().hashCode(); @@ -23975,53 +24041,53 @@ public final class ClientProtos { return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom(java.io.InputStream input) throws java.io.IOException { return PARSER.parseFrom(input); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return PARSER.parseFrom(input, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return PARSER.parseDelimitedFrom(input); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return PARSER.parseDelimitedFrom(input, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return PARSER.parseFrom(input); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -24030,7 +24096,7 @@ public final class ClientProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -24042,30 +24108,28 @@ public final class ClientProtos { return builder; } /** - * Protobuf type {@code MultiAction} + * Protobuf type {@code Action} * *
-     **
-     * An action that is part of MultiRequest.
-     * This is a union type - exactly one of the fields will be set.
+     * Either a Get or a Mutation
      * 
*/ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiAction_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Action_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiAction_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Action_fieldAccessorTable .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder.class); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder.class); } - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -24087,18 +24151,20 @@ public final class ClientProtos { public Builder clear() { super.clear(); + index_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); if (mutationBuilder_ == null) { mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); } else { mutationBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000001); + bitField0_ = (bitField0_ & ~0x00000002); if (getBuilder_ == null) { get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance(); } else { getBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -24108,35 +24174,39 @@ public final class ClientProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiAction_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_Action_descriptor; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction(this); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } + result.index_ = index_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } if (mutationBuilder_ == null) { result.mutation_ = mutation_; } else { result.mutation_ = mutationBuilder_.build(); } - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; } if (getBuilder_ == null) { result.get_ = get_; @@ -24149,16 +24219,19 @@ public final class ClientProtos { } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.getDefaultInstance()) return this; + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.getDefaultInstance()) return this; + if (other.hasIndex()) { + setIndex(other.getIndex()); + } if (other.hasMutation()) { mergeMutation(other.getMutation()); } @@ -24189,11 +24262,11 @@ public final class ClientProtos { com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction parsedMessage = null; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction) e.getUnfinishedMessage(); + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action) e.getUnfinishedMessage(); throw e; } finally { if (parsedMessage != null) { @@ -24204,18 +24277,71 @@ public final class ClientProtos { } private int bitField0_; - // optional .MutationProto mutation = 1; + // optional uint32 index = 1; + private int index_ ; + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi action, useful aligning
+       * result with what was originally submitted.
+       * 
+ */ + public boolean hasIndex() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi action, useful aligning
+       * result with what was originally submitted.
+       * 
+ */ + public int getIndex() { + return index_; + } + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi action, useful aligning
+       * result with what was originally submitted.
+       * 
+ */ + public Builder setIndex(int value) { + bitField0_ |= 0x00000001; + index_ = value; + onChanged(); + return this; + } + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi action, useful aligning
+       * result with what was originally submitted.
+       * 
+ */ + public Builder clearIndex() { + bitField0_ = (bitField0_ & ~0x00000001); + index_ = 0; + onChanged(); + return this; + } + + // optional .MutationProto mutation = 2; private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder> mutationBuilder_; /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public boolean hasMutation() { - return ((bitField0_ & 0x00000001) == 0x00000001); + return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutation() { if (mutationBuilder_ == null) { @@ -24225,7 +24351,7 @@ public final class ClientProtos { } } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public Builder setMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { if (mutationBuilder_ == null) { @@ -24237,11 +24363,11 @@ public final class ClientProtos { } else { mutationBuilder_.setMessage(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public Builder setMutation( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) { @@ -24251,15 +24377,15 @@ public final class ClientProtos { } else { mutationBuilder_.setMessage(builderForValue.build()); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public Builder mergeMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { if (mutationBuilder_ == null) { - if (((bitField0_ & 0x00000001) == 0x00000001) && + if (((bitField0_ & 0x00000002) == 0x00000002) && mutation_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance()) { mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder(mutation_).mergeFrom(value).buildPartial(); @@ -24270,11 +24396,11 @@ public final class ClientProtos { } else { mutationBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public Builder clearMutation() { if (mutationBuilder_ == null) { @@ -24283,19 +24409,19 @@ public final class ClientProtos { } else { mutationBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000001); + bitField0_ = (bitField0_ & ~0x00000002); return this; } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder getMutationBuilder() { - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; onChanged(); return getMutationFieldBuilder().getBuilder(); } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationOrBuilder() { if (mutationBuilder_ != null) { @@ -24305,7 +24431,7 @@ public final class ClientProtos { } } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder> @@ -24321,18 +24447,18 @@ public final class ClientProtos { return mutationBuilder_; } - // optional .Get get = 2; + // optional .Get get = 3; private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder> getBuilder_; /** - * optional .Get get = 2; + * optional .Get get = 3; */ public boolean hasGet() { - return ((bitField0_ & 0x00000002) == 0x00000002); + return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get getGet() { if (getBuilder_ == null) { @@ -24342,7 +24468,7 @@ public final class ClientProtos { } } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public Builder setGet(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get value) { if (getBuilder_ == null) { @@ -24354,11 +24480,11 @@ public final class ClientProtos { } else { getBuilder_.setMessage(value); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public Builder setGet( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder builderForValue) { @@ -24368,15 +24494,15 @@ public final class ClientProtos { } else { getBuilder_.setMessage(builderForValue.build()); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public Builder mergeGet(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get value) { if (getBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002) && + if (((bitField0_ & 0x00000004) == 0x00000004) && get_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance()) { get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.newBuilder(get_).mergeFrom(value).buildPartial(); @@ -24387,11 +24513,11 @@ public final class ClientProtos { } else { getBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public Builder clearGet() { if (getBuilder_ == null) { @@ -24400,19 +24526,19 @@ public final class ClientProtos { } else { getBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000004); return this; } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder getGetBuilder() { - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; onChanged(); return getGetFieldBuilder().getBuilder(); } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder() { if (getBuilder_ != null) { @@ -24422,7 +24548,7 @@ public final class ClientProtos { } } /** - * optional .Get get = 2; + * optional .Get get = 3; */ private com.google.protobuf.SingleFieldBuilder< org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder> @@ -24438,77 +24564,101 @@ public final class ClientProtos { return getBuilder_; } - // @@protoc_insertion_point(builder_scope:MultiAction) + // @@protoc_insertion_point(builder_scope:Action) } static { - defaultInstance = new MultiAction(true); + defaultInstance = new Action(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:MultiAction) + // @@protoc_insertion_point(class_scope:Action) } - public interface ActionResultOrBuilder + public interface RegionActionOrBuilder extends com.google.protobuf.MessageOrBuilder { - // optional .Result value = 1; + // required .RegionSpecifier region = 1; /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - boolean hasValue(); + boolean hasRegion(); + /** + * required .RegionSpecifier region = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); + /** + * required .RegionSpecifier region = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); + + // optional bool atomic = 2; /** - * optional .Result value = 1; + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
*/ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getValue(); + boolean hasAtomic(); /** - * optional .Result value = 1; + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
*/ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getValueOrBuilder(); + boolean getAtomic(); - // optional .NameBytesPair exception = 2; + // repeated .Action action = 3; /** - * optional .NameBytesPair exception = 2; + * repeated .Action action = 3; */ - boolean hasException(); + java.util.List + getActionList(); /** - * optional .NameBytesPair exception = 2; + * repeated .Action action = 3; */ - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action getAction(int index); /** - * optional .NameBytesPair exception = 2; + * repeated .Action action = 3; */ - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder(); + int getActionCount(); + /** + * repeated .Action action = 3; + */ + java.util.List + getActionOrBuilderList(); + /** + * repeated .Action action = 3; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder getActionOrBuilder( + int index); } /** - * Protobuf type {@code ActionResult} + * Protobuf type {@code RegionAction} * *
    **
-   * An individual action result. The result will in the
-   * same order as the action in the request. If an action
-   * returns a value, it is set in value field. If it doesn't
-   * return anything, the result will be empty. If an action
-   * fails to execute due to any exception, the exception
-   * is returned as a stringified parameter.
+   * Actions to run against a Region.
    * 
*/ - public static final class ActionResult extends + public static final class RegionAction extends com.google.protobuf.GeneratedMessage - implements ActionResultOrBuilder { - // Use ActionResult.newBuilder() to construct. - private ActionResult(com.google.protobuf.GeneratedMessage.Builder builder) { + implements RegionActionOrBuilder { + // Use RegionAction.newBuilder() to construct. + private RegionAction(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); this.unknownFields = builder.getUnknownFields(); } - private ActionResult(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private RegionAction(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - private static final ActionResult defaultInstance; - public static ActionResult getDefaultInstance() { + private static final RegionAction defaultInstance; + public static RegionAction getDefaultInstance() { return defaultInstance; } - public ActionResult getDefaultInstanceForType() { + public RegionAction getDefaultInstanceForType() { return defaultInstance; } @@ -24518,7 +24668,7 @@ public final class ClientProtos { getUnknownFields() { return this.unknownFields; } - private ActionResult( + private RegionAction( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -24542,29 +24692,29 @@ public final class ClientProtos { break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder subBuilder = null; + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null; if (((bitField0_ & 0x00000001) == 0x00000001)) { - subBuilder = value_.toBuilder(); + subBuilder = region_.toBuilder(); } - value_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.PARSER, extensionRegistry); + region_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry); if (subBuilder != null) { - subBuilder.mergeFrom(value_); - value_ = subBuilder.buildPartial(); + subBuilder.mergeFrom(region_); + region_ = subBuilder.buildPartial(); } bitField0_ |= 0x00000001; break; } - case 18: { - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) == 0x00000002)) { - subBuilder = exception_.toBuilder(); - } - exception_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(exception_); - exception_ = subBuilder.buildPartial(); - } + case 16: { bitField0_ |= 0x00000002; + atomic_ = input.readBool(); + break; + } + case 26: { + if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + action_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000004; + } + action_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.PARSER, extensionRegistry)); break; } } @@ -24575,97 +24725,147 @@ public final class ClientProtos { throw new com.google.protobuf.InvalidProtocolBufferException( e.getMessage()).setUnfinishedMessage(this); } finally { + if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + action_ = java.util.Collections.unmodifiableList(action_); + } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); } } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ActionResult_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionAction_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ActionResult_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionAction_fieldAccessorTable .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder.class); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder.class); } - public static com.google.protobuf.Parser PARSER = - new com.google.protobuf.AbstractParser() { - public ActionResult parsePartialFrom( + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public RegionAction parsePartialFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - return new ActionResult(input, extensionRegistry); + return new RegionAction(input, extensionRegistry); } }; @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public com.google.protobuf.Parser getParserForType() { return PARSER; } private int bitField0_; - // optional .Result value = 1; - public static final int VALUE_FIELD_NUMBER = 1; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value_; + // required .RegionSpecifier region = 1; + public static final int REGION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_; /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public boolean hasValue() { + public boolean hasRegion() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getValue() { - return value_; + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { + return region_; } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getValueOrBuilder() { - return value_; + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { + return region_; } - // optional .NameBytesPair exception = 2; - public static final int EXCEPTION_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair exception_; + // optional bool atomic = 2; + public static final int ATOMIC_FIELD_NUMBER = 2; + private boolean atomic_; /** - * optional .NameBytesPair exception = 2; + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
*/ - public boolean hasException() { + public boolean hasAtomic() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .NameBytesPair exception = 2; + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { - return exception_; + public boolean getAtomic() { + return atomic_; + } + + // repeated .Action action = 3; + public static final int ACTION_FIELD_NUMBER = 3; + private java.util.List action_; + /** + * repeated .Action action = 3; + */ + public java.util.List getActionList() { + return action_; } /** - * optional .NameBytesPair exception = 2; + * repeated .Action action = 3; */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { - return exception_; + public java.util.List + getActionOrBuilderList() { + return action_; + } + /** + * repeated .Action action = 3; + */ + public int getActionCount() { + return action_.size(); + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action getAction(int index) { + return action_.get(index); + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder getActionOrBuilder( + int index) { + return action_.get(index); } private void initFields() { - value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); - exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + atomic_ = false; + action_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (hasException()) { - if (!getException().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } + if (!hasRegion()) { + memoizedIsInitialized = 0; + return false; + } + if (!getRegion().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getActionCount(); i++) { + if (!getAction(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -24674,10 +24874,13 @@ public final class ClientProtos { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, value_); + output.writeMessage(1, region_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, exception_); + output.writeBool(2, atomic_); + } + for (int i = 0; i < action_.size(); i++) { + output.writeMessage(3, action_.get(i)); } getUnknownFields().writeTo(output); } @@ -24690,11 +24893,15 @@ public final class ClientProtos { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, value_); + .computeMessageSize(1, region_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, exception_); + .computeBoolSize(2, atomic_); + } + for (int i = 0; i < action_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, action_.get(i)); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -24713,22 +24920,24 @@ public final class ClientProtos { if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult) obj; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction) obj; boolean result = true; - result = result && (hasValue() == other.hasValue()); - if (hasValue()) { - result = result && getValue() - .equals(other.getValue()); + result = result && (hasRegion() == other.hasRegion()); + if (hasRegion()) { + result = result && getRegion() + .equals(other.getRegion()); } - result = result && (hasException() == other.hasException()); - if (hasException()) { - result = result && getException() - .equals(other.getException()); + result = result && (hasAtomic() == other.hasAtomic()); + if (hasAtomic()) { + result = result && (getAtomic() + == other.getAtomic()); } + result = result && getActionList() + .equals(other.getActionList()); result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -24742,66 +24951,70 @@ public final class ClientProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasValue()) { - hash = (37 * hash) + VALUE_FIELD_NUMBER; - hash = (53 * hash) + getValue().hashCode(); + if (hasRegion()) { + hash = (37 * hash) + REGION_FIELD_NUMBER; + hash = (53 * hash) + getRegion().hashCode(); } - if (hasException()) { - hash = (37 * hash) + EXCEPTION_FIELD_NUMBER; - hash = (53 * hash) + getException().hashCode(); + if (hasAtomic()) { + hash = (37 * hash) + ATOMIC_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getAtomic()); + } + if (getActionCount() > 0) { + hash = (37 * hash) + ACTION_FIELD_NUMBER; + hash = (53 * hash) + getActionList().hashCode(); } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom(byte[] data) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return PARSER.parseFrom(data, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom(java.io.InputStream input) throws java.io.IOException { return PARSER.parseFrom(input); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return PARSER.parseFrom(input, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseDelimitedFrom(java.io.InputStream input) + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return PARSER.parseDelimitedFrom(input); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseDelimitedFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return PARSER.parseDelimitedFrom(input, extensionRegistry); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return PARSER.parseFrom(input); } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -24810,7 +25023,7 @@ public final class ClientProtos { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -24822,34 +25035,29 @@ public final class ClientProtos { return builder; } /** - * Protobuf type {@code ActionResult} + * Protobuf type {@code RegionAction} * *
      **
-     * An individual action result. The result will in the
-     * same order as the action in the request. If an action
-     * returns a value, it is set in value field. If it doesn't
-     * return anything, the result will be empty. If an action
-     * fails to execute due to any exception, the exception
-     * is returned as a stringified parameter.
+     * Actions to run against a Region.
      * 
*/ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder { + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ActionResult_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionAction_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ActionResult_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionAction_fieldAccessorTable .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder.class); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder.class); } - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -24861,8 +25069,8 @@ public final class ClientProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getValueFieldBuilder(); - getExceptionFieldBuilder(); + getRegionFieldBuilder(); + getActionFieldBuilder(); } } private static Builder create() { @@ -24871,18 +25079,20 @@ public final class ClientProtos { public Builder clear() { super.clear(); - if (valueBuilder_ == null) { - value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); } else { - valueBuilder_.clear(); + regionBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - if (exceptionBuilder_ == null) { - exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + atomic_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + if (actionBuilder_ == null) { + action_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); } else { - exceptionBuilder_.clear(); + actionBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -24892,40 +25102,45 @@ public final class ClientProtos { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ActionResult_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionAction_descriptor; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.getDefaultInstance(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.getDefaultInstance(); } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult result = buildPartial(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult(this); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - if (valueBuilder_ == null) { - result.value_ = value_; + if (regionBuilder_ == null) { + result.region_ = region_; } else { - result.value_ = valueBuilder_.build(); + result.region_ = regionBuilder_.build(); } if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - if (exceptionBuilder_ == null) { - result.exception_ = exception_; + result.atomic_ = atomic_; + if (actionBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { + action_ = java.util.Collections.unmodifiableList(action_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.action_ = action_; } else { - result.exception_ = exceptionBuilder_.build(); + result.action_ = actionBuilder_.build(); } result.bitField0_ = to_bitField0_; onBuilt(); @@ -24933,29 +25148,63 @@ public final class ClientProtos { } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult)other); + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.getDefaultInstance()) return this; - if (other.hasValue()) { - mergeValue(other.getValue()); + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.getDefaultInstance()) return this; + if (other.hasRegion()) { + mergeRegion(other.getRegion()); } - if (other.hasException()) { - mergeException(other.getException()); + if (other.hasAtomic()) { + setAtomic(other.getAtomic()); + } + if (actionBuilder_ == null) { + if (!other.action_.isEmpty()) { + if (action_.isEmpty()) { + action_ = other.action_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureActionIsMutable(); + action_.addAll(other.action_); + } + onChanged(); + } + } else { + if (!other.action_.isEmpty()) { + if (actionBuilder_.isEmpty()) { + actionBuilder_.dispose(); + actionBuilder_ = null; + action_ = other.action_; + bitField0_ = (bitField0_ & ~0x00000004); + actionBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getActionFieldBuilder() : null; + } else { + actionBuilder_.addAllMessages(other.action_); + } + } } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (hasException()) { - if (!getException().isInitialized()) { + if (!hasRegion()) { + + return false; + } + if (!getRegion().isInitialized()) { + + return false; + } + for (int i = 0; i < getActionCount(); i++) { + if (!getAction(i).isInitialized()) { return false; } @@ -24967,11 +25216,11 @@ public final class ClientProtos { com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parsedMessage = null; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult) e.getUnfinishedMessage(); + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction) e.getUnfinishedMessage(); throw e; } finally { if (parsedMessage != null) { @@ -24982,313 +25231,2090 @@ public final class ClientProtos { } private int bitField0_; - // optional .Result value = 1; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + // required .RegionSpecifier region = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder> valueBuilder_; + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_; /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public boolean hasValue() { + public boolean hasRegion() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getValue() { - if (valueBuilder_ == null) { - return value_; + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { + if (regionBuilder_ == null) { + return region_; } else { - return valueBuilder_.getMessage(); + return regionBuilder_.getMessage(); } } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public Builder setValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (valueBuilder_ == null) { + public Builder setRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - value_ = value; + region_ = value; onChanged(); } else { - valueBuilder_.setMessage(value); + regionBuilder_.setMessage(value); } bitField0_ |= 0x00000001; return this; } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public Builder setValue( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder builderForValue) { - if (valueBuilder_ == null) { - value_ = builderForValue.build(); + public Builder setRegion( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) { + if (regionBuilder_ == null) { + region_ = builderForValue.build(); onChanged(); } else { - valueBuilder_.setMessage(builderForValue.build()); + regionBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000001; return this; } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public Builder mergeValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (valueBuilder_ == null) { + public Builder mergeRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionBuilder_ == null) { if (((bitField0_ & 0x00000001) == 0x00000001) && - value_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance()) { - value_ = - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.newBuilder(value_).mergeFrom(value).buildPartial(); + region_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) { + region_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(region_).mergeFrom(value).buildPartial(); } else { - value_ = value; + region_ = value; } onChanged(); } else { - valueBuilder_.mergeFrom(value); + regionBuilder_.mergeFrom(value); } bitField0_ |= 0x00000001; return this; } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public Builder clearValue() { - if (valueBuilder_ == null) { - value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + public Builder clearRegion() { + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); onChanged(); } else { - valueBuilder_.clear(); + regionBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); return this; } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder getValueBuilder() { + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() { bitField0_ |= 0x00000001; onChanged(); - return getValueFieldBuilder().getBuilder(); + return getRegionFieldBuilder().getBuilder(); } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getValueOrBuilder() { - if (valueBuilder_ != null) { - return valueBuilder_.getMessageOrBuilder(); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { + if (regionBuilder_ != null) { + return regionBuilder_.getMessageOrBuilder(); } else { - return value_; + return region_; } } /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder> - getValueFieldBuilder() { - if (valueBuilder_ == null) { - valueBuilder_ = new com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder>( - value_, + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> + getRegionFieldBuilder() { + if (regionBuilder_ == null) { + regionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>( + region_, getParentForChildren(), isClean()); - value_ = null; + region_ = null; } - return valueBuilder_; + return regionBuilder_; } - // optional .NameBytesPair exception = 2; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> exceptionBuilder_; + // optional bool atomic = 2; + private boolean atomic_ ; /** - * optional .NameBytesPair exception = 2; + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
*/ - public boolean hasException() { + public boolean hasAtomic() { return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * optional .NameBytesPair exception = 2; + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { - if (exceptionBuilder_ == null) { - return exception_; - } else { - return exceptionBuilder_.getMessage(); - } + public boolean getAtomic() { + return atomic_; } /** - * optional .NameBytesPair exception = 2; + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
*/ - public Builder setException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { - if (exceptionBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - exception_ = value; - onChanged(); - } else { - exceptionBuilder_.setMessage(value); - } + public Builder setAtomic(boolean value) { bitField0_ |= 0x00000002; + atomic_ = value; + onChanged(); return this; } /** - * optional .NameBytesPair exception = 2; - */ + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
+ */ + public Builder clearAtomic() { + bitField0_ = (bitField0_ & ~0x00000002); + atomic_ = false; + onChanged(); + return this; + } + + // repeated .Action action = 3; + private java.util.List action_ = + java.util.Collections.emptyList(); + private void ensureActionIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + action_ = new java.util.ArrayList(action_); + bitField0_ |= 0x00000004; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder> actionBuilder_; + + /** + * repeated .Action action = 3; + */ + public java.util.List getActionList() { + if (actionBuilder_ == null) { + return java.util.Collections.unmodifiableList(action_); + } else { + return actionBuilder_.getMessageList(); + } + } + /** + * repeated .Action action = 3; + */ + public int getActionCount() { + if (actionBuilder_ == null) { + return action_.size(); + } else { + return actionBuilder_.getCount(); + } + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action getAction(int index) { + if (actionBuilder_ == null) { + return action_.get(index); + } else { + return actionBuilder_.getMessage(index); + } + } + /** + * repeated .Action action = 3; + */ + public Builder setAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action value) { + if (actionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureActionIsMutable(); + action_.set(index, value); + onChanged(); + } else { + actionBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder setAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder builderForValue) { + if (actionBuilder_ == null) { + ensureActionIsMutable(); + action_.set(index, builderForValue.build()); + onChanged(); + } else { + actionBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder addAction(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action value) { + if (actionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureActionIsMutable(); + action_.add(value); + onChanged(); + } else { + actionBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder addAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action value) { + if (actionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureActionIsMutable(); + action_.add(index, value); + onChanged(); + } else { + actionBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder addAction( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder builderForValue) { + if (actionBuilder_ == null) { + ensureActionIsMutable(); + action_.add(builderForValue.build()); + onChanged(); + } else { + actionBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder addAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder builderForValue) { + if (actionBuilder_ == null) { + ensureActionIsMutable(); + action_.add(index, builderForValue.build()); + onChanged(); + } else { + actionBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder addAllAction( + java.lang.Iterable values) { + if (actionBuilder_ == null) { + ensureActionIsMutable(); + super.addAll(values, action_); + onChanged(); + } else { + actionBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder clearAction() { + if (actionBuilder_ == null) { + action_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + } else { + actionBuilder_.clear(); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public Builder removeAction(int index) { + if (actionBuilder_ == null) { + ensureActionIsMutable(); + action_.remove(index); + onChanged(); + } else { + actionBuilder_.remove(index); + } + return this; + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder getActionBuilder( + int index) { + return getActionFieldBuilder().getBuilder(index); + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder getActionOrBuilder( + int index) { + if (actionBuilder_ == null) { + return action_.get(index); } else { + return actionBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .Action action = 3; + */ + public java.util.List + getActionOrBuilderList() { + if (actionBuilder_ != null) { + return actionBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(action_); + } + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder addActionBuilder() { + return getActionFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.getDefaultInstance()); + } + /** + * repeated .Action action = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder addActionBuilder( + int index) { + return getActionFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.getDefaultInstance()); + } + /** + * repeated .Action action = 3; + */ + public java.util.List + getActionBuilderList() { + return getActionFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder> + getActionFieldBuilder() { + if (actionBuilder_ == null) { + actionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionOrBuilder>( + action_, + ((bitField0_ & 0x00000004) == 0x00000004), + getParentForChildren(), + isClean()); + action_ = null; + } + return actionBuilder_; + } + + // @@protoc_insertion_point(builder_scope:RegionAction) + } + + static { + defaultInstance = new RegionAction(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RegionAction) + } + + public interface ResultOrExceptionOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional uint32 index = 1; + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi call, save original index of the list of all
+     * passed so can align this response w/ original request.
+     * 
+ */ + boolean hasIndex(); + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi call, save original index of the list of all
+     * passed so can align this response w/ original request.
+     * 
+ */ + int getIndex(); + + // optional .Result result = 2; + /** + * optional .Result result = 2; + */ + boolean hasResult(); + /** + * optional .Result result = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResult(); + /** + * optional .Result result = 2; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder(); + + // optional .NameBytesPair exception = 3; + /** + * optional .NameBytesPair exception = 3; + */ + boolean hasException(); + /** + * optional .NameBytesPair exception = 3; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException(); + /** + * optional .NameBytesPair exception = 3; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder(); + } + /** + * Protobuf type {@code ResultOrException} + * + *
+   **
+   * Either a Result or an Exception NameBytesPair (keyed by
+   * exception name whose value is the exception stringified)
+   * or maybe empty if no result and no exception.
+   * 
+ */ + public static final class ResultOrException extends + com.google.protobuf.GeneratedMessage + implements ResultOrExceptionOrBuilder { + // Use ResultOrException.newBuilder() to construct. + private ResultOrException(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ResultOrException(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ResultOrException defaultInstance; + public static ResultOrException getDefaultInstance() { + return defaultInstance; + } + + public ResultOrException getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ResultOrException( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + index_ = input.readUInt32(); + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = result_.toBuilder(); + } + result_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(result_); + result_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + case 26: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + subBuilder = exception_.toBuilder(); + } + exception_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(exception_); + exception_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ResultOrException_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ResultOrException_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ResultOrException parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ResultOrException(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional uint32 index = 1; + public static final int INDEX_FIELD_NUMBER = 1; + private int index_; + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi call, save original index of the list of all
+     * passed so can align this response w/ original request.
+     * 
+ */ + public boolean hasIndex() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint32 index = 1; + * + *
+     * If part of a multi call, save original index of the list of all
+     * passed so can align this response w/ original request.
+     * 
+ */ + public int getIndex() { + return index_; + } + + // optional .Result result = 2; + public static final int RESULT_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result result_; + /** + * optional .Result result = 2; + */ + public boolean hasResult() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .Result result = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResult() { + return result_; + } + /** + * optional .Result result = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder() { + return result_; + } + + // optional .NameBytesPair exception = 3; + public static final int EXCEPTION_FIELD_NUMBER = 3; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair exception_; + /** + * optional .NameBytesPair exception = 3; + */ + public boolean hasException() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .NameBytesPair exception = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { + return exception_; + } + /** + * optional .NameBytesPair exception = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { + return exception_; + } + + private void initFields() { + index_ = 0; + result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasException()) { + if (!getException().isInitialized()) { + 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.writeUInt32(1, index_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, result_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, exception_); + } + 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 + .computeUInt32Size(1, index_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, result_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, exception_); + } + 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.protobuf.generated.ClientProtos.ResultOrException)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException) obj; + + boolean result = true; + result = result && (hasIndex() == other.hasIndex()); + if (hasIndex()) { + result = result && (getIndex() + == other.getIndex()); + } + result = result && (hasResult() == other.hasResult()); + if (hasResult()) { + result = result && getResult() + .equals(other.getResult()); + } + result = result && (hasException() == other.hasException()); + if (hasException()) { + result = result && getException() + .equals(other.getException()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasIndex()) { + hash = (37 * hash) + INDEX_FIELD_NUMBER; + hash = (53 * hash) + getIndex(); + } + if (hasResult()) { + hash = (37 * hash) + RESULT_FIELD_NUMBER; + hash = (53 * hash) + getResult().hashCode(); + } + if (hasException()) { + hash = (37 * hash) + EXCEPTION_FIELD_NUMBER; + hash = (53 * hash) + getException().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException 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; + } + /** + * Protobuf type {@code ResultOrException} + * + *
+     **
+     * Either a Result or an Exception NameBytesPair (keyed by
+     * exception name whose value is the exception stringified)
+     * or maybe empty if no result and no exception.
+     * 
+ */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ResultOrException_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ResultOrException_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getResultFieldBuilder(); + getExceptionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + index_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + if (resultBuilder_ == null) { + result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + } else { + resultBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + if (exceptionBuilder_ == null) { + exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + } else { + exceptionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_ResultOrException_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.index_ = index_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (resultBuilder_ == null) { + result.result_ = result_; + } else { + result.result_ = resultBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (exceptionBuilder_ == null) { + result.exception_ = exception_; + } else { + result.exception_ = exceptionBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.getDefaultInstance()) return this; + if (other.hasIndex()) { + setIndex(other.getIndex()); + } + if (other.hasResult()) { + mergeResult(other.getResult()); + } + if (other.hasException()) { + mergeException(other.getException()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasException()) { + if (!getException().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional uint32 index = 1; + private int index_ ; + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi call, save original index of the list of all
+       * passed so can align this response w/ original request.
+       * 
+ */ + public boolean hasIndex() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi call, save original index of the list of all
+       * passed so can align this response w/ original request.
+       * 
+ */ + public int getIndex() { + return index_; + } + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi call, save original index of the list of all
+       * passed so can align this response w/ original request.
+       * 
+ */ + public Builder setIndex(int value) { + bitField0_ |= 0x00000001; + index_ = value; + onChanged(); + return this; + } + /** + * optional uint32 index = 1; + * + *
+       * If part of a multi call, save original index of the list of all
+       * passed so can align this response w/ original request.
+       * 
+ */ + public Builder clearIndex() { + bitField0_ = (bitField0_ & ~0x00000001); + index_ = 0; + onChanged(); + return this; + } + + // optional .Result result = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder> resultBuilder_; + /** + * optional .Result result = 2; + */ + public boolean hasResult() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .Result result = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResult() { + if (resultBuilder_ == null) { + return result_; + } else { + return resultBuilder_.getMessage(); + } + } + /** + * optional .Result result = 2; + */ + public Builder setResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { + if (resultBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + result_ = value; + onChanged(); + } else { + resultBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * optional .Result result = 2; + */ + public Builder setResult( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder builderForValue) { + if (resultBuilder_ == null) { + result_ = builderForValue.build(); + onChanged(); + } else { + resultBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * optional .Result result = 2; + */ + public Builder mergeResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { + if (resultBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + result_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance()) { + result_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.newBuilder(result_).mergeFrom(value).buildPartial(); + } else { + result_ = value; + } + onChanged(); + } else { + resultBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * optional .Result result = 2; + */ + public Builder clearResult() { + if (resultBuilder_ == null) { + result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + onChanged(); + } else { + resultBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * optional .Result result = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder getResultBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getResultFieldBuilder().getBuilder(); + } + /** + * optional .Result result = 2; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder() { + if (resultBuilder_ != null) { + return resultBuilder_.getMessageOrBuilder(); + } else { + return result_; + } + } + /** + * optional .Result result = 2; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder> + getResultFieldBuilder() { + if (resultBuilder_ == null) { + resultBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder>( + result_, + getParentForChildren(), + isClean()); + result_ = null; + } + return resultBuilder_; + } + + // optional .NameBytesPair exception = 3; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> exceptionBuilder_; + /** + * optional .NameBytesPair exception = 3; + */ + public boolean hasException() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .NameBytesPair exception = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { + if (exceptionBuilder_ == null) { + return exception_; + } else { + return exceptionBuilder_.getMessage(); + } + } + /** + * optional .NameBytesPair exception = 3; + */ + public Builder setException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { + if (exceptionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + exception_ = value; + onChanged(); + } else { + exceptionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .NameBytesPair exception = 3; + */ public Builder setException( org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) { if (exceptionBuilder_ == null) { exception_ = builderForValue.build(); onChanged(); } else { - exceptionBuilder_.setMessage(builderForValue.build()); + exceptionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .NameBytesPair exception = 3; + */ + public Builder mergeException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { + if (exceptionBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + exception_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance()) { + exception_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(exception_).mergeFrom(value).buildPartial(); + } else { + exception_ = value; + } + onChanged(); + } else { + exceptionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .NameBytesPair exception = 3; + */ + public Builder clearException() { + if (exceptionBuilder_ == null) { + exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + onChanged(); + } else { + exceptionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + /** + * optional .NameBytesPair exception = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getExceptionBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getExceptionFieldBuilder().getBuilder(); + } + /** + * optional .NameBytesPair exception = 3; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { + if (exceptionBuilder_ != null) { + return exceptionBuilder_.getMessageOrBuilder(); + } else { + return exception_; + } + } + /** + * optional .NameBytesPair exception = 3; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> + getExceptionFieldBuilder() { + if (exceptionBuilder_ == null) { + exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>( + exception_, + getParentForChildren(), + isClean()); + exception_ = null; + } + return exceptionBuilder_; + } + + // @@protoc_insertion_point(builder_scope:ResultOrException) + } + + static { + defaultInstance = new ResultOrException(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ResultOrException) + } + + public interface RegionActionResultOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .ResultOrException resultOrException = 1; + /** + * repeated .ResultOrException resultOrException = 1; + */ + java.util.List + getResultOrExceptionList(); + /** + * repeated .ResultOrException resultOrException = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException getResultOrException(int index); + /** + * repeated .ResultOrException resultOrException = 1; + */ + int getResultOrExceptionCount(); + /** + * repeated .ResultOrException resultOrException = 1; + */ + java.util.List + getResultOrExceptionOrBuilderList(); + /** + * repeated .ResultOrException resultOrException = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder getResultOrExceptionOrBuilder( + int index); + } + /** + * Protobuf type {@code RegionActionResult} + * + *
+   **
+   * The result of a RegionAction.
+   * 
+ */ + public static final class RegionActionResult extends + com.google.protobuf.GeneratedMessage + implements RegionActionResultOrBuilder { + // Use RegionActionResult.newBuilder() to construct. + private RegionActionResult(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private RegionActionResult(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final RegionActionResult defaultInstance; + public static RegionActionResult getDefaultInstance() { + return defaultInstance; + } + + public RegionActionResult getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RegionActionResult( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + resultOrException_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + resultOrException_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.PARSER, extensionRegistry)); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + resultOrException_ = java.util.Collections.unmodifiableList(resultOrException_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public RegionActionResult parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RegionActionResult(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + // repeated .ResultOrException resultOrException = 1; + public static final int RESULTOREXCEPTION_FIELD_NUMBER = 1; + private java.util.List resultOrException_; + /** + * repeated .ResultOrException resultOrException = 1; + */ + public java.util.List getResultOrExceptionList() { + return resultOrException_; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public java.util.List + getResultOrExceptionOrBuilderList() { + return resultOrException_; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public int getResultOrExceptionCount() { + return resultOrException_.size(); + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException getResultOrException(int index) { + return resultOrException_.get(index); + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder getResultOrExceptionOrBuilder( + int index) { + return resultOrException_.get(index); + } + + private void initFields() { + resultOrException_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + for (int i = 0; i < getResultOrExceptionCount(); i++) { + if (!getResultOrException(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + for (int i = 0; i < resultOrException_.size(); i++) { + output.writeMessage(1, resultOrException_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < resultOrException_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, resultOrException_.get(i)); + } + 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.protobuf.generated.ClientProtos.RegionActionResult)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult) obj; + + boolean result = true; + result = result && getResultOrExceptionList() + .equals(other.getResultOrExceptionList()); + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getResultOrExceptionCount() > 0) { + hash = (37 * hash) + RESULTOREXCEPTION_FIELD_NUMBER; + hash = (53 * hash) + getResultOrExceptionList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult 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; + } + /** + * Protobuf type {@code RegionActionResult} + * + *
+     **
+     * The result of a RegionAction.
+     * 
+ */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getResultOrExceptionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (resultOrExceptionBuilder_ == null) { + resultOrException_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + resultOrExceptionBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult(this); + int from_bitField0_ = bitField0_; + if (resultOrExceptionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + resultOrException_ = java.util.Collections.unmodifiableList(resultOrException_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.resultOrException_ = resultOrException_; + } else { + result.resultOrException_ = resultOrExceptionBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.getDefaultInstance()) return this; + if (resultOrExceptionBuilder_ == null) { + if (!other.resultOrException_.isEmpty()) { + if (resultOrException_.isEmpty()) { + resultOrException_ = other.resultOrException_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureResultOrExceptionIsMutable(); + resultOrException_.addAll(other.resultOrException_); + } + onChanged(); + } + } else { + if (!other.resultOrException_.isEmpty()) { + if (resultOrExceptionBuilder_.isEmpty()) { + resultOrExceptionBuilder_.dispose(); + resultOrExceptionBuilder_ = null; + resultOrException_ = other.resultOrException_; + bitField0_ = (bitField0_ & ~0x00000001); + resultOrExceptionBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getResultOrExceptionFieldBuilder() : null; + } else { + resultOrExceptionBuilder_.addAllMessages(other.resultOrException_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getResultOrExceptionCount(); i++) { + if (!getResultOrException(i).isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // repeated .ResultOrException resultOrException = 1; + private java.util.List resultOrException_ = + java.util.Collections.emptyList(); + private void ensureResultOrExceptionIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + resultOrException_ = new java.util.ArrayList(resultOrException_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder> resultOrExceptionBuilder_; + + /** + * repeated .ResultOrException resultOrException = 1; + */ + public java.util.List getResultOrExceptionList() { + if (resultOrExceptionBuilder_ == null) { + return java.util.Collections.unmodifiableList(resultOrException_); + } else { + return resultOrExceptionBuilder_.getMessageList(); + } + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public int getResultOrExceptionCount() { + if (resultOrExceptionBuilder_ == null) { + return resultOrException_.size(); + } else { + return resultOrExceptionBuilder_.getCount(); + } + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException getResultOrException(int index) { + if (resultOrExceptionBuilder_ == null) { + return resultOrException_.get(index); + } else { + return resultOrExceptionBuilder_.getMessage(index); + } + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public Builder setResultOrException( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException value) { + if (resultOrExceptionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureResultOrExceptionIsMutable(); + resultOrException_.set(index, value); + onChanged(); + } else { + resultOrExceptionBuilder_.setMessage(index, value); } - bitField0_ |= 0x00000002; return this; } /** - * optional .NameBytesPair exception = 2; + * repeated .ResultOrException resultOrException = 1; */ - public Builder mergeException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { - if (exceptionBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - exception_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance()) { - exception_ = - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(exception_).mergeFrom(value).buildPartial(); - } else { - exception_ = value; + public Builder setResultOrException( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder builderForValue) { + if (resultOrExceptionBuilder_ == null) { + ensureResultOrExceptionIsMutable(); + resultOrException_.set(index, builderForValue.build()); + onChanged(); + } else { + resultOrExceptionBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public Builder addResultOrException(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException value) { + if (resultOrExceptionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); } + ensureResultOrExceptionIsMutable(); + resultOrException_.add(value); onChanged(); } else { - exceptionBuilder_.mergeFrom(value); + resultOrExceptionBuilder_.addMessage(value); } - bitField0_ |= 0x00000002; return this; } /** - * optional .NameBytesPair exception = 2; + * repeated .ResultOrException resultOrException = 1; */ - public Builder clearException() { - if (exceptionBuilder_ == null) { - exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + public Builder addResultOrException( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException value) { + if (resultOrExceptionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureResultOrExceptionIsMutable(); + resultOrException_.add(index, value); onChanged(); } else { - exceptionBuilder_.clear(); + resultOrExceptionBuilder_.addMessage(index, value); } - bitField0_ = (bitField0_ & ~0x00000002); return this; } /** - * optional .NameBytesPair exception = 2; + * repeated .ResultOrException resultOrException = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getExceptionBuilder() { - bitField0_ |= 0x00000002; - onChanged(); - return getExceptionFieldBuilder().getBuilder(); + public Builder addResultOrException( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder builderForValue) { + if (resultOrExceptionBuilder_ == null) { + ensureResultOrExceptionIsMutable(); + resultOrException_.add(builderForValue.build()); + onChanged(); + } else { + resultOrExceptionBuilder_.addMessage(builderForValue.build()); + } + return this; } /** - * optional .NameBytesPair exception = 2; + * repeated .ResultOrException resultOrException = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { - if (exceptionBuilder_ != null) { - return exceptionBuilder_.getMessageOrBuilder(); + public Builder addResultOrException( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder builderForValue) { + if (resultOrExceptionBuilder_ == null) { + ensureResultOrExceptionIsMutable(); + resultOrException_.add(index, builderForValue.build()); + onChanged(); } else { - return exception_; + resultOrExceptionBuilder_.addMessage(index, builderForValue.build()); } + return this; } /** - * optional .NameBytesPair exception = 2; + * repeated .ResultOrException resultOrException = 1; */ - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> - getExceptionFieldBuilder() { - if (exceptionBuilder_ == null) { - exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>( - exception_, + public Builder addAllResultOrException( + java.lang.Iterable values) { + if (resultOrExceptionBuilder_ == null) { + ensureResultOrExceptionIsMutable(); + super.addAll(values, resultOrException_); + onChanged(); + } else { + resultOrExceptionBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public Builder clearResultOrException() { + if (resultOrExceptionBuilder_ == null) { + resultOrException_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + resultOrExceptionBuilder_.clear(); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public Builder removeResultOrException(int index) { + if (resultOrExceptionBuilder_ == null) { + ensureResultOrExceptionIsMutable(); + resultOrException_.remove(index); + onChanged(); + } else { + resultOrExceptionBuilder_.remove(index); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder getResultOrExceptionBuilder( + int index) { + return getResultOrExceptionFieldBuilder().getBuilder(index); + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder getResultOrExceptionOrBuilder( + int index) { + if (resultOrExceptionBuilder_ == null) { + return resultOrException_.get(index); } else { + return resultOrExceptionBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public java.util.List + getResultOrExceptionOrBuilderList() { + if (resultOrExceptionBuilder_ != null) { + return resultOrExceptionBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(resultOrException_); + } + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder addResultOrExceptionBuilder() { + return getResultOrExceptionFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.getDefaultInstance()); + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder addResultOrExceptionBuilder( + int index) { + return getResultOrExceptionFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.getDefaultInstance()); + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + public java.util.List + getResultOrExceptionBuilderList() { + return getResultOrExceptionFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder> + getResultOrExceptionFieldBuilder() { + if (resultOrExceptionBuilder_ == null) { + resultOrExceptionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder>( + resultOrException_, + ((bitField0_ & 0x00000001) == 0x00000001), getParentForChildren(), isClean()); - exception_ = null; + resultOrException_ = null; } - return exceptionBuilder_; + return resultOrExceptionBuilder_; } - // @@protoc_insertion_point(builder_scope:ActionResult) + // @@protoc_insertion_point(builder_scope:RegionActionResult) } static { - defaultInstance = new ActionResult(true); + defaultInstance = new RegionActionResult(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:ActionResult) + // @@protoc_insertion_point(class_scope:RegionActionResult) } public interface MultiRequestOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required .RegionSpecifier region = 1; - /** - * required .RegionSpecifier region = 1; - */ - boolean hasRegion(); - /** - * required .RegionSpecifier region = 1; - */ - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); - /** - * required .RegionSpecifier region = 1; - */ - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); - - // repeated .MultiAction action = 2; + // repeated .RegionAction regionAction = 1; /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - java.util.List - getActionList(); + java.util.List + getRegionActionList(); /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction getAction(int index); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction getRegionAction(int index); /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - int getActionCount(); + int getRegionActionCount(); /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - java.util.List - getActionOrBuilderList(); + java.util.List + getRegionActionOrBuilderList(); /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder getActionOrBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder( int index); - - // optional bool atomic = 3; - /** - * optional bool atomic = 3; - */ - boolean hasAtomic(); - /** - * optional bool atomic = 3; - */ - boolean getAtomic(); } /** * Protobuf type {@code MultiRequest} * *
    **
-   * You can execute a list of actions on a given region in order.
-   *
-   * If it is a list of mutate actions, atomic can be set
-   * to make sure they can be processed atomically, just like
-   * RowMutations.
+   * Execute a list of actions on a given region in order.
    * 
*/ public static final class MultiRequest extends @@ -25340,29 +27366,11 @@ public final class ClientProtos { break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - subBuilder = region_.toBuilder(); - } - region_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry); - if (subBuilder != null) { - subBuilder.mergeFrom(region_); - region_ = subBuilder.buildPartial(); - } - bitField0_ |= 0x00000001; - break; - } - case 18: { - if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) { - action_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + regionAction_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; } - action_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.PARSER, extensionRegistry)); - break; - } - case 24: { - bitField0_ |= 0x00000002; - atomic_ = input.readBool(); + regionAction_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.PARSER, extensionRegistry)); break; } } @@ -25373,8 +27381,8 @@ public final class ClientProtos { throw new com.google.protobuf.InvalidProtocolBufferException( e.getMessage()).setUnfinishedMessage(this); } finally { - if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { - action_ = java.util.Collections.unmodifiableList(action_); + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + regionAction_ = java.util.Collections.unmodifiableList(regionAction_); } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); @@ -25407,101 +27415,52 @@ public final class ClientProtos { return PARSER; } - private int bitField0_; - // required .RegionSpecifier region = 1; - public static final int REGION_FIELD_NUMBER = 1; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_; - /** - * required .RegionSpecifier region = 1; - */ - public boolean hasRegion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - /** - * required .RegionSpecifier region = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { - return region_; - } - /** - * required .RegionSpecifier region = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { - return region_; - } - - // repeated .MultiAction action = 2; - public static final int ACTION_FIELD_NUMBER = 2; - private java.util.List action_; + // repeated .RegionAction regionAction = 1; + public static final int REGIONACTION_FIELD_NUMBER = 1; + private java.util.List regionAction_; /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public java.util.List getActionList() { - return action_; + public java.util.List getRegionActionList() { + return regionAction_; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public java.util.List - getActionOrBuilderList() { - return action_; + public java.util.List + getRegionActionOrBuilderList() { + return regionAction_; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public int getActionCount() { - return action_.size(); + public int getRegionActionCount() { + return regionAction_.size(); } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction getAction(int index) { - return action_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction getRegionAction(int index) { + return regionAction_.get(index); } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder getActionOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder( int index) { - return action_.get(index); - } - - // optional bool atomic = 3; - public static final int ATOMIC_FIELD_NUMBER = 3; - private boolean atomic_; - /** - * optional bool atomic = 3; - */ - public boolean hasAtomic() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - /** - * optional bool atomic = 3; - */ - public boolean getAtomic() { - return atomic_; + return regionAction_.get(index); } - - private void initFields() { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - action_ = java.util.Collections.emptyList(); - atomic_ = false; + + private void initFields() { + regionAction_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasRegion()) { - memoizedIsInitialized = 0; - return false; - } - if (!getRegion().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getActionCount(); i++) { - if (!getAction(i).isInitialized()) { + for (int i = 0; i < getRegionActionCount(); i++) { + if (!getRegionAction(i).isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -25513,14 +27472,8 @@ public final class ClientProtos { public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, region_); - } - for (int i = 0; i < action_.size(); i++) { - output.writeMessage(2, action_.get(i)); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBool(3, atomic_); + for (int i = 0; i < regionAction_.size(); i++) { + output.writeMessage(1, regionAction_.get(i)); } getUnknownFields().writeTo(output); } @@ -25531,17 +27484,9 @@ public final class ClientProtos { if (size != -1) return size; size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, region_); - } - for (int i = 0; i < action_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, action_.get(i)); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { + for (int i = 0; i < regionAction_.size(); i++) { size += com.google.protobuf.CodedOutputStream - .computeBoolSize(3, atomic_); + .computeMessageSize(1, regionAction_.get(i)); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -25566,18 +27511,8 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest) obj; boolean result = true; - result = result && (hasRegion() == other.hasRegion()); - if (hasRegion()) { - result = result && getRegion() - .equals(other.getRegion()); - } - result = result && getActionList() - .equals(other.getActionList()); - result = result && (hasAtomic() == other.hasAtomic()); - if (hasAtomic()) { - result = result && (getAtomic() - == other.getAtomic()); - } + result = result && getRegionActionList() + .equals(other.getRegionActionList()); result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -25591,17 +27526,9 @@ public final class ClientProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasRegion()) { - hash = (37 * hash) + REGION_FIELD_NUMBER; - hash = (53 * hash) + getRegion().hashCode(); - } - if (getActionCount() > 0) { - hash = (37 * hash) + ACTION_FIELD_NUMBER; - hash = (53 * hash) + getActionList().hashCode(); - } - if (hasAtomic()) { - hash = (37 * hash) + ATOMIC_FIELD_NUMBER; - hash = (53 * hash) + hashBoolean(getAtomic()); + if (getRegionActionCount() > 0) { + hash = (37 * hash) + REGIONACTION_FIELD_NUMBER; + hash = (53 * hash) + getRegionActionList().hashCode(); } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; @@ -25679,11 +27606,7 @@ public final class ClientProtos { * *
      **
-     * You can execute a list of actions on a given region in order.
-     *
-     * If it is a list of mutate actions, atomic can be set
-     * to make sure they can be processed atomically, just like
-     * RowMutations.
+     * Execute a list of actions on a given region in order.
      * 
*/ public static final class Builder extends @@ -25713,8 +27636,7 @@ public final class ClientProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getRegionFieldBuilder(); - getActionFieldBuilder(); + getRegionActionFieldBuilder(); } } private static Builder create() { @@ -25723,20 +27645,12 @@ public final class ClientProtos { public Builder clear() { super.clear(); - if (regionBuilder_ == null) { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - } else { - regionBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000001); - if (actionBuilder_ == null) { - action_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); + if (regionActionBuilder_ == null) { + regionAction_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); } else { - actionBuilder_.clear(); + regionActionBuilder_.clear(); } - atomic_ = false; - bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -25764,29 +27678,15 @@ public final class ClientProtos { public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest buildPartial() { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest(this); int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - if (regionBuilder_ == null) { - result.region_ = region_; - } else { - result.region_ = regionBuilder_.build(); - } - if (actionBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002)) { - action_ = java.util.Collections.unmodifiableList(action_); - bitField0_ = (bitField0_ & ~0x00000002); + if (regionActionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + regionAction_ = java.util.Collections.unmodifiableList(regionAction_); + bitField0_ = (bitField0_ & ~0x00000001); } - result.action_ = action_; + result.regionAction_ = regionAction_; } else { - result.action_ = actionBuilder_.build(); - } - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000002; + result.regionAction_ = regionActionBuilder_.build(); } - result.atomic_ = atomic_; - result.bitField0_ = to_bitField0_; onBuilt(); return result; } @@ -25802,467 +27702,303 @@ public final class ClientProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest other) { if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance()) return this; - if (other.hasRegion()) { - mergeRegion(other.getRegion()); - } - if (actionBuilder_ == null) { - if (!other.action_.isEmpty()) { - if (action_.isEmpty()) { - action_ = other.action_; - bitField0_ = (bitField0_ & ~0x00000002); + if (regionActionBuilder_ == null) { + if (!other.regionAction_.isEmpty()) { + if (regionAction_.isEmpty()) { + regionAction_ = other.regionAction_; + bitField0_ = (bitField0_ & ~0x00000001); } else { - ensureActionIsMutable(); - action_.addAll(other.action_); + ensureRegionActionIsMutable(); + regionAction_.addAll(other.regionAction_); } onChanged(); } } else { - if (!other.action_.isEmpty()) { - if (actionBuilder_.isEmpty()) { - actionBuilder_.dispose(); - actionBuilder_ = null; - action_ = other.action_; - bitField0_ = (bitField0_ & ~0x00000002); - actionBuilder_ = + if (!other.regionAction_.isEmpty()) { + if (regionActionBuilder_.isEmpty()) { + regionActionBuilder_.dispose(); + regionActionBuilder_ = null; + regionAction_ = other.regionAction_; + bitField0_ = (bitField0_ & ~0x00000001); + regionActionBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getActionFieldBuilder() : null; + getRegionActionFieldBuilder() : null; } else { - actionBuilder_.addAllMessages(other.action_); + regionActionBuilder_.addAllMessages(other.regionAction_); } } } - if (other.hasAtomic()) { - setAtomic(other.getAtomic()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasRegion()) { - - return false; - } - if (!getRegion().isInitialized()) { - - return false; - } - for (int i = 0; i < getActionCount(); i++) { - if (!getAction(i).isInitialized()) { + for (int i = 0; i < getRegionActionCount(); i++) { + if (!getRegionAction(i).isInitialized()) { return false; } } - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest) e.getUnfinishedMessage(); - throw e; - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - // required .RegionSpecifier region = 1; - private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_; - /** - * required .RegionSpecifier region = 1; - */ - public boolean hasRegion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - /** - * required .RegionSpecifier region = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { - if (regionBuilder_ == null) { - return region_; - } else { - return regionBuilder_.getMessage(); - } - } - /** - * required .RegionSpecifier region = 1; - */ - public Builder setRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { - if (regionBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - region_ = value; - onChanged(); - } else { - regionBuilder_.setMessage(value); - } - bitField0_ |= 0x00000001; - return this; - } - /** - * required .RegionSpecifier region = 1; - */ - public Builder setRegion( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) { - if (regionBuilder_ == null) { - region_ = builderForValue.build(); - onChanged(); - } else { - regionBuilder_.setMessage(builderForValue.build()); - } - bitField0_ |= 0x00000001; - return this; - } - /** - * required .RegionSpecifier region = 1; - */ - public Builder mergeRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { - if (regionBuilder_ == null) { - if (((bitField0_ & 0x00000001) == 0x00000001) && - region_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) { - region_ = - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(region_).mergeFrom(value).buildPartial(); - } else { - region_ = value; - } - onChanged(); - } else { - regionBuilder_.mergeFrom(value); - } - bitField0_ |= 0x00000001; - return this; - } - /** - * required .RegionSpecifier region = 1; - */ - public Builder clearRegion() { - if (regionBuilder_ == null) { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - onChanged(); - } else { - regionBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - /** - * required .RegionSpecifier region = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() { - bitField0_ |= 0x00000001; - onChanged(); - return getRegionFieldBuilder().getBuilder(); - } - /** - * required .RegionSpecifier region = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { - if (regionBuilder_ != null) { - return regionBuilder_.getMessageOrBuilder(); - } else { - return region_; - } - } - /** - * required .RegionSpecifier region = 1; - */ - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> - getRegionFieldBuilder() { - if (regionBuilder_ == null) { - regionBuilder_ = new com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>( - region_, - getParentForChildren(), - isClean()); - region_ = null; + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } } - return regionBuilder_; + return this; } + private int bitField0_; - // repeated .MultiAction action = 2; - private java.util.List action_ = + // repeated .RegionAction regionAction = 1; + private java.util.List regionAction_ = java.util.Collections.emptyList(); - private void ensureActionIsMutable() { - if (!((bitField0_ & 0x00000002) == 0x00000002)) { - action_ = new java.util.ArrayList(action_); - bitField0_ |= 0x00000002; + private void ensureRegionActionIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + regionAction_ = new java.util.ArrayList(regionAction_); + bitField0_ |= 0x00000001; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder> actionBuilder_; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder> regionActionBuilder_; /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public java.util.List getActionList() { - if (actionBuilder_ == null) { - return java.util.Collections.unmodifiableList(action_); + public java.util.List getRegionActionList() { + if (regionActionBuilder_ == null) { + return java.util.Collections.unmodifiableList(regionAction_); } else { - return actionBuilder_.getMessageList(); + return regionActionBuilder_.getMessageList(); } } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public int getActionCount() { - if (actionBuilder_ == null) { - return action_.size(); + public int getRegionActionCount() { + if (regionActionBuilder_ == null) { + return regionAction_.size(); } else { - return actionBuilder_.getCount(); + return regionActionBuilder_.getCount(); } } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction getAction(int index) { - if (actionBuilder_ == null) { - return action_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction getRegionAction(int index) { + if (regionActionBuilder_ == null) { + return regionAction_.get(index); } else { - return actionBuilder_.getMessage(index); + return regionActionBuilder_.getMessage(index); } } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder setAction( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction value) { - if (actionBuilder_ == null) { + public Builder setRegionAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction value) { + if (regionActionBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureActionIsMutable(); - action_.set(index, value); + ensureRegionActionIsMutable(); + regionAction_.set(index, value); onChanged(); } else { - actionBuilder_.setMessage(index, value); + regionActionBuilder_.setMessage(index, value); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder setAction( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder builderForValue) { - if (actionBuilder_ == null) { - ensureActionIsMutable(); - action_.set(index, builderForValue.build()); + public Builder setRegionAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder builderForValue) { + if (regionActionBuilder_ == null) { + ensureRegionActionIsMutable(); + regionAction_.set(index, builderForValue.build()); onChanged(); } else { - actionBuilder_.setMessage(index, builderForValue.build()); + regionActionBuilder_.setMessage(index, builderForValue.build()); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder addAction(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction value) { - if (actionBuilder_ == null) { + public Builder addRegionAction(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction value) { + if (regionActionBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureActionIsMutable(); - action_.add(value); + ensureRegionActionIsMutable(); + regionAction_.add(value); onChanged(); } else { - actionBuilder_.addMessage(value); + regionActionBuilder_.addMessage(value); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder addAction( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction value) { - if (actionBuilder_ == null) { + public Builder addRegionAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction value) { + if (regionActionBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureActionIsMutable(); - action_.add(index, value); + ensureRegionActionIsMutable(); + regionAction_.add(index, value); onChanged(); } else { - actionBuilder_.addMessage(index, value); + regionActionBuilder_.addMessage(index, value); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder addAction( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder builderForValue) { - if (actionBuilder_ == null) { - ensureActionIsMutable(); - action_.add(builderForValue.build()); + public Builder addRegionAction( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder builderForValue) { + if (regionActionBuilder_ == null) { + ensureRegionActionIsMutable(); + regionAction_.add(builderForValue.build()); onChanged(); } else { - actionBuilder_.addMessage(builderForValue.build()); + regionActionBuilder_.addMessage(builderForValue.build()); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder addAction( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder builderForValue) { - if (actionBuilder_ == null) { - ensureActionIsMutable(); - action_.add(index, builderForValue.build()); + public Builder addRegionAction( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder builderForValue) { + if (regionActionBuilder_ == null) { + ensureRegionActionIsMutable(); + regionAction_.add(index, builderForValue.build()); onChanged(); } else { - actionBuilder_.addMessage(index, builderForValue.build()); + regionActionBuilder_.addMessage(index, builderForValue.build()); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder addAllAction( - java.lang.Iterable values) { - if (actionBuilder_ == null) { - ensureActionIsMutable(); - super.addAll(values, action_); + public Builder addAllRegionAction( + java.lang.Iterable values) { + if (regionActionBuilder_ == null) { + ensureRegionActionIsMutable(); + super.addAll(values, regionAction_); onChanged(); } else { - actionBuilder_.addAllMessages(values); + regionActionBuilder_.addAllMessages(values); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder clearAction() { - if (actionBuilder_ == null) { - action_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); + public Builder clearRegionAction() { + if (regionActionBuilder_ == null) { + regionAction_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); onChanged(); } else { - actionBuilder_.clear(); + regionActionBuilder_.clear(); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public Builder removeAction(int index) { - if (actionBuilder_ == null) { - ensureActionIsMutable(); - action_.remove(index); + public Builder removeRegionAction(int index) { + if (regionActionBuilder_ == null) { + ensureRegionActionIsMutable(); + regionAction_.remove(index); onChanged(); } else { - actionBuilder_.remove(index); + regionActionBuilder_.remove(index); } return this; } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder getActionBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder getRegionActionBuilder( int index) { - return getActionFieldBuilder().getBuilder(index); + return getRegionActionFieldBuilder().getBuilder(index); } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder getActionOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder( int index) { - if (actionBuilder_ == null) { - return action_.get(index); } else { - return actionBuilder_.getMessageOrBuilder(index); + if (regionActionBuilder_ == null) { + return regionAction_.get(index); } else { + return regionActionBuilder_.getMessageOrBuilder(index); } } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public java.util.List - getActionOrBuilderList() { - if (actionBuilder_ != null) { - return actionBuilder_.getMessageOrBuilderList(); + public java.util.List + getRegionActionOrBuilderList() { + if (regionActionBuilder_ != null) { + return regionActionBuilder_.getMessageOrBuilderList(); } else { - return java.util.Collections.unmodifiableList(action_); + return java.util.Collections.unmodifiableList(regionAction_); } } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder addActionBuilder() { - return getActionFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.getDefaultInstance()); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder addRegionActionBuilder() { + return getRegionActionFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.getDefaultInstance()); } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder addActionBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder addRegionActionBuilder( int index) { - return getActionFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.getDefaultInstance()); + return getRegionActionFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.getDefaultInstance()); } /** - * repeated .MultiAction action = 2; + * repeated .RegionAction regionAction = 1; */ - public java.util.List - getActionBuilderList() { - return getActionFieldBuilder().getBuilderList(); + public java.util.List + getRegionActionBuilderList() { + return getRegionActionFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder> - getActionFieldBuilder() { - if (actionBuilder_ == null) { - actionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder>( - action_, - ((bitField0_ & 0x00000002) == 0x00000002), + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder> + getRegionActionFieldBuilder() { + if (regionActionBuilder_ == null) { + regionActionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder>( + regionAction_, + ((bitField0_ & 0x00000001) == 0x00000001), getParentForChildren(), isClean()); - action_ = null; + regionAction_ = null; } - return actionBuilder_; - } - - // optional bool atomic = 3; - private boolean atomic_ ; - /** - * optional bool atomic = 3; - */ - public boolean hasAtomic() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - /** - * optional bool atomic = 3; - */ - public boolean getAtomic() { - return atomic_; - } - /** - * optional bool atomic = 3; - */ - public Builder setAtomic(boolean value) { - bitField0_ |= 0x00000004; - atomic_ = value; - onChanged(); - return this; - } - /** - * optional bool atomic = 3; - */ - public Builder clearAtomic() { - bitField0_ = (bitField0_ & ~0x00000004); - atomic_ = false; - onChanged(); - return this; + return regionActionBuilder_; } // @@protoc_insertion_point(builder_scope:MultiRequest) @@ -26279,29 +28015,29 @@ public final class ClientProtos { public interface MultiResponseOrBuilder extends com.google.protobuf.MessageOrBuilder { - // repeated .ActionResult result = 1; + // repeated .RegionActionResult regionActionResult = 1; /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - java.util.List - getResultList(); + java.util.List + getRegionActionResultList(); /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult getResult(int index); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult getRegionActionResult(int index); /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - int getResultCount(); + int getRegionActionResultCount(); /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - java.util.List - getResultOrBuilderList(); + java.util.List + getRegionActionResultOrBuilderList(); /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder getResultOrBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder getRegionActionResultOrBuilder( int index); } /** @@ -26357,10 +28093,10 @@ public final class ClientProtos { } case 10: { if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { - result_ = new java.util.ArrayList(); + regionActionResult_ = new java.util.ArrayList(); mutable_bitField0_ |= 0x00000001; } - result_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.PARSER, extensionRegistry)); + regionActionResult_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.PARSER, extensionRegistry)); break; } } @@ -26372,7 +28108,7 @@ public final class ClientProtos { e.getMessage()).setUnfinishedMessage(this); } finally { if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { - result_ = java.util.Collections.unmodifiableList(result_); + regionActionResult_ = java.util.Collections.unmodifiableList(regionActionResult_); } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); @@ -26405,52 +28141,52 @@ public final class ClientProtos { return PARSER; } - // repeated .ActionResult result = 1; - public static final int RESULT_FIELD_NUMBER = 1; - private java.util.List result_; + // repeated .RegionActionResult regionActionResult = 1; + public static final int REGIONACTIONRESULT_FIELD_NUMBER = 1; + private java.util.List regionActionResult_; /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public java.util.List getResultList() { - return result_; + public java.util.List getRegionActionResultList() { + return regionActionResult_; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public java.util.List - getResultOrBuilderList() { - return result_; + public java.util.List + getRegionActionResultOrBuilderList() { + return regionActionResult_; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public int getResultCount() { - return result_.size(); + public int getRegionActionResultCount() { + return regionActionResult_.size(); } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult getResult(int index) { - return result_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult getRegionActionResult(int index) { + return regionActionResult_.get(index); } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder getResultOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder getRegionActionResultOrBuilder( int index) { - return result_.get(index); + return regionActionResult_.get(index); } private void initFields() { - result_ = java.util.Collections.emptyList(); + regionActionResult_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { + for (int i = 0; i < getRegionActionResultCount(); i++) { + if (!getRegionActionResult(i).isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -26462,8 +28198,8 @@ public final class ClientProtos { public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - for (int i = 0; i < result_.size(); i++) { - output.writeMessage(1, result_.get(i)); + for (int i = 0; i < regionActionResult_.size(); i++) { + output.writeMessage(1, regionActionResult_.get(i)); } getUnknownFields().writeTo(output); } @@ -26474,9 +28210,9 @@ public final class ClientProtos { if (size != -1) return size; size = 0; - for (int i = 0; i < result_.size(); i++) { + for (int i = 0; i < regionActionResult_.size(); i++) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, result_.get(i)); + .computeMessageSize(1, regionActionResult_.get(i)); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -26501,8 +28237,8 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse) obj; boolean result = true; - result = result && getResultList() - .equals(other.getResultList()); + result = result && getRegionActionResultList() + .equals(other.getRegionActionResultList()); result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -26516,9 +28252,9 @@ public final class ClientProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (getResultCount() > 0) { - hash = (37 * hash) + RESULT_FIELD_NUMBER; - hash = (53 * hash) + getResultList().hashCode(); + if (getRegionActionResultCount() > 0) { + hash = (37 * hash) + REGIONACTIONRESULT_FIELD_NUMBER; + hash = (53 * hash) + getRegionActionResultList().hashCode(); } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; @@ -26621,7 +28357,7 @@ public final class ClientProtos { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getResultFieldBuilder(); + getRegionActionResultFieldBuilder(); } } private static Builder create() { @@ -26630,11 +28366,11 @@ public final class ClientProtos { public Builder clear() { super.clear(); - if (resultBuilder_ == null) { - result_ = java.util.Collections.emptyList(); + if (regionActionResultBuilder_ == null) { + regionActionResult_ = java.util.Collections.emptyList(); bitField0_ = (bitField0_ & ~0x00000001); } else { - resultBuilder_.clear(); + regionActionResultBuilder_.clear(); } return this; } @@ -26663,14 +28399,14 @@ public final class ClientProtos { public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse buildPartial() { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse(this); int from_bitField0_ = bitField0_; - if (resultBuilder_ == null) { + if (regionActionResultBuilder_ == null) { if (((bitField0_ & 0x00000001) == 0x00000001)) { - result_ = java.util.Collections.unmodifiableList(result_); + regionActionResult_ = java.util.Collections.unmodifiableList(regionActionResult_); bitField0_ = (bitField0_ & ~0x00000001); } - result.result_ = result_; + result.regionActionResult_ = regionActionResult_; } else { - result.result_ = resultBuilder_.build(); + result.regionActionResult_ = regionActionResultBuilder_.build(); } onBuilt(); return result; @@ -26687,29 +28423,29 @@ public final class ClientProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse other) { if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance()) return this; - if (resultBuilder_ == null) { - if (!other.result_.isEmpty()) { - if (result_.isEmpty()) { - result_ = other.result_; + if (regionActionResultBuilder_ == null) { + if (!other.regionActionResult_.isEmpty()) { + if (regionActionResult_.isEmpty()) { + regionActionResult_ = other.regionActionResult_; bitField0_ = (bitField0_ & ~0x00000001); } else { - ensureResultIsMutable(); - result_.addAll(other.result_); + ensureRegionActionResultIsMutable(); + regionActionResult_.addAll(other.regionActionResult_); } onChanged(); } } else { - if (!other.result_.isEmpty()) { - if (resultBuilder_.isEmpty()) { - resultBuilder_.dispose(); - resultBuilder_ = null; - result_ = other.result_; + if (!other.regionActionResult_.isEmpty()) { + if (regionActionResultBuilder_.isEmpty()) { + regionActionResultBuilder_.dispose(); + regionActionResultBuilder_ = null; + regionActionResult_ = other.regionActionResult_; bitField0_ = (bitField0_ & ~0x00000001); - resultBuilder_ = + regionActionResultBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getResultFieldBuilder() : null; + getRegionActionResultFieldBuilder() : null; } else { - resultBuilder_.addAllMessages(other.result_); + regionActionResultBuilder_.addAllMessages(other.regionActionResult_); } } } @@ -26718,8 +28454,8 @@ public final class ClientProtos { } public final boolean isInitialized() { - for (int i = 0; i < getResultCount(); i++) { - if (!getResult(i).isInitialized()) { + for (int i = 0; i < getRegionActionResultCount(); i++) { + if (!getRegionActionResult(i).isInitialized()) { return false; } @@ -26746,244 +28482,244 @@ public final class ClientProtos { } private int bitField0_; - // repeated .ActionResult result = 1; - private java.util.List result_ = + // repeated .RegionActionResult regionActionResult = 1; + private java.util.List regionActionResult_ = java.util.Collections.emptyList(); - private void ensureResultIsMutable() { + private void ensureRegionActionResultIsMutable() { if (!((bitField0_ & 0x00000001) == 0x00000001)) { - result_ = new java.util.ArrayList(result_); + regionActionResult_ = new java.util.ArrayList(regionActionResult_); bitField0_ |= 0x00000001; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder> resultBuilder_; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder> regionActionResultBuilder_; /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public java.util.List getResultList() { - if (resultBuilder_ == null) { - return java.util.Collections.unmodifiableList(result_); + public java.util.List getRegionActionResultList() { + if (regionActionResultBuilder_ == null) { + return java.util.Collections.unmodifiableList(regionActionResult_); } else { - return resultBuilder_.getMessageList(); + return regionActionResultBuilder_.getMessageList(); } } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public int getResultCount() { - if (resultBuilder_ == null) { - return result_.size(); + public int getRegionActionResultCount() { + if (regionActionResultBuilder_ == null) { + return regionActionResult_.size(); } else { - return resultBuilder_.getCount(); + return regionActionResultBuilder_.getCount(); } } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult getResult(int index) { - if (resultBuilder_ == null) { - return result_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult getRegionActionResult(int index) { + if (regionActionResultBuilder_ == null) { + return regionActionResult_.get(index); } else { - return resultBuilder_.getMessage(index); + return regionActionResultBuilder_.getMessage(index); } } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder setResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult value) { - if (resultBuilder_ == null) { + public Builder setRegionActionResult( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult value) { + if (regionActionResultBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureResultIsMutable(); - result_.set(index, value); + ensureRegionActionResultIsMutable(); + regionActionResult_.set(index, value); onChanged(); } else { - resultBuilder_.setMessage(index, value); + regionActionResultBuilder_.setMessage(index, value); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder setResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder builderForValue) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.set(index, builderForValue.build()); + public Builder setRegionActionResult( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder builderForValue) { + if (regionActionResultBuilder_ == null) { + ensureRegionActionResultIsMutable(); + regionActionResult_.set(index, builderForValue.build()); onChanged(); } else { - resultBuilder_.setMessage(index, builderForValue.build()); + regionActionResultBuilder_.setMessage(index, builderForValue.build()); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder addResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult value) { - if (resultBuilder_ == null) { + public Builder addRegionActionResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult value) { + if (regionActionResultBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureResultIsMutable(); - result_.add(value); + ensureRegionActionResultIsMutable(); + regionActionResult_.add(value); onChanged(); } else { - resultBuilder_.addMessage(value); + regionActionResultBuilder_.addMessage(value); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder addResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult value) { - if (resultBuilder_ == null) { + public Builder addRegionActionResult( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult value) { + if (regionActionResultBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - ensureResultIsMutable(); - result_.add(index, value); + ensureRegionActionResultIsMutable(); + regionActionResult_.add(index, value); onChanged(); } else { - resultBuilder_.addMessage(index, value); + regionActionResultBuilder_.addMessage(index, value); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder addResult( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder builderForValue) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.add(builderForValue.build()); + public Builder addRegionActionResult( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder builderForValue) { + if (regionActionResultBuilder_ == null) { + ensureRegionActionResultIsMutable(); + regionActionResult_.add(builderForValue.build()); onChanged(); } else { - resultBuilder_.addMessage(builderForValue.build()); + regionActionResultBuilder_.addMessage(builderForValue.build()); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder addResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder builderForValue) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.add(index, builderForValue.build()); + public Builder addRegionActionResult( + int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder builderForValue) { + if (regionActionResultBuilder_ == null) { + ensureRegionActionResultIsMutable(); + regionActionResult_.add(index, builderForValue.build()); onChanged(); } else { - resultBuilder_.addMessage(index, builderForValue.build()); + regionActionResultBuilder_.addMessage(index, builderForValue.build()); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder addAllResult( - java.lang.Iterable values) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - super.addAll(values, result_); + public Builder addAllRegionActionResult( + java.lang.Iterable values) { + if (regionActionResultBuilder_ == null) { + ensureRegionActionResultIsMutable(); + super.addAll(values, regionActionResult_); onChanged(); } else { - resultBuilder_.addAllMessages(values); + regionActionResultBuilder_.addAllMessages(values); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder clearResult() { - if (resultBuilder_ == null) { - result_ = java.util.Collections.emptyList(); + public Builder clearRegionActionResult() { + if (regionActionResultBuilder_ == null) { + regionActionResult_ = java.util.Collections.emptyList(); bitField0_ = (bitField0_ & ~0x00000001); onChanged(); } else { - resultBuilder_.clear(); + regionActionResultBuilder_.clear(); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public Builder removeResult(int index) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.remove(index); + public Builder removeRegionActionResult(int index) { + if (regionActionResultBuilder_ == null) { + ensureRegionActionResultIsMutable(); + regionActionResult_.remove(index); onChanged(); } else { - resultBuilder_.remove(index); + regionActionResultBuilder_.remove(index); } return this; } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder getResultBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder getRegionActionResultBuilder( int index) { - return getResultFieldBuilder().getBuilder(index); + return getRegionActionResultFieldBuilder().getBuilder(index); } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder getResultOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder getRegionActionResultOrBuilder( int index) { - if (resultBuilder_ == null) { - return result_.get(index); } else { - return resultBuilder_.getMessageOrBuilder(index); + if (regionActionResultBuilder_ == null) { + return regionActionResult_.get(index); } else { + return regionActionResultBuilder_.getMessageOrBuilder(index); } } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public java.util.List - getResultOrBuilderList() { - if (resultBuilder_ != null) { - return resultBuilder_.getMessageOrBuilderList(); + public java.util.List + getRegionActionResultOrBuilderList() { + if (regionActionResultBuilder_ != null) { + return regionActionResultBuilder_.getMessageOrBuilderList(); } else { - return java.util.Collections.unmodifiableList(result_); + return java.util.Collections.unmodifiableList(regionActionResult_); } } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder addResultBuilder() { - return getResultFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.getDefaultInstance()); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder addRegionActionResultBuilder() { + return getRegionActionResultFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.getDefaultInstance()); } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder addResultBuilder( + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder addRegionActionResultBuilder( int index) { - return getResultFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.getDefaultInstance()); + return getRegionActionResultFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.getDefaultInstance()); } /** - * repeated .ActionResult result = 1; + * repeated .RegionActionResult regionActionResult = 1; */ - public java.util.List - getResultBuilderList() { - return getResultFieldBuilder().getBuilderList(); + public java.util.List + getRegionActionResultBuilderList() { + return getRegionActionResultFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder> - getResultFieldBuilder() { - if (resultBuilder_ == null) { - resultBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResultOrBuilder>( - result_, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder> + getRegionActionResultFieldBuilder() { + if (regionActionResultBuilder_ == null) { + regionActionResultBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResultOrBuilder>( + regionActionResult_, ((bitField0_ & 0x00000001) == 0x00000001), getParentForChildren(), isClean()); - result_ = null; + regionActionResult_ = null; } - return resultBuilder_; + return regionActionResultBuilder_; } // @@protoc_insertion_point(builder_scope:MultiResponse) @@ -27766,15 +29502,25 @@ public final class ClientProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_CoprocessorServiceResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_MultiAction_descriptor; + internal_static_Action_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Action_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RegionAction_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RegionAction_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_ResultOrException_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_MultiAction_fieldAccessorTable; + internal_static_ResultOrException_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_ActionResult_descriptor; + internal_static_RegionActionResult_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_ActionResult_fieldAccessorTable; + internal_static_RegionActionResult_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_MultiRequest_descriptor; private static @@ -27870,25 +29616,29 @@ public final class ClientProtos { "\n\004call\030\002 \002(\0132\027.CoprocessorServiceCall\"]\n" + "\032CoprocessorServiceResponse\022 \n\006region\030\001 " + "\002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(\0132\016.N" + - "ameBytesPair\"B\n\013MultiAction\022 \n\010mutation\030" + - "\001 \001(\0132\016.MutationProto\022\021\n\003get\030\002 \001(\0132\004.Get" + - "\"I\n\014ActionResult\022\026\n\005value\030\001 \001(\0132\007.Result" + - "\022!\n\texception\030\002 \001(\0132\016.NameBytesPair\"^\n\014M", - "ultiRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpec" + - "ifier\022\034\n\006action\030\002 \003(\0132\014.MultiAction\022\016\n\006a" + - "tomic\030\003 \001(\010\".\n\rMultiResponse\022\035\n\006result\030\001" + - " \003(\0132\r.ActionResult2\342\002\n\rClientService\022 \n" + - "\003Get\022\013.GetRequest\032\014.GetResponse\022/\n\010Multi" + - "Get\022\020.MultiGetRequest\032\021.MultiGetResponse" + - "\022)\n\006Mutate\022\016.MutateRequest\032\017.MutateRespo" + - "nse\022#\n\004Scan\022\014.ScanRequest\032\r.ScanResponse" + - "\022>\n\rBulkLoadHFile\022\025.BulkLoadHFileRequest" + - "\032\026.BulkLoadHFileResponse\022F\n\013ExecService\022", - "\032.CoprocessorServiceRequest\032\033.Coprocesso" + - "rServiceResponse\022&\n\005Multi\022\r.MultiRequest" + - "\032\016.MultiResponseBB\n*org.apache.hadoop.hb" + - "ase.protobuf.generatedB\014ClientProtosH\001\210\001" + - "\001\240\001\001" + "ameBytesPair\"L\n\006Action\022\r\n\005index\030\001 \001(\r\022 \n" + + "\010mutation\030\002 \001(\0132\016.MutationProto\022\021\n\003get\030\003" + + " \001(\0132\004.Get\"Y\n\014RegionAction\022 \n\006region\030\001 \002" + + "(\0132\020.RegionSpecifier\022\016\n\006atomic\030\002 \001(\010\022\027\n\006", + "action\030\003 \003(\0132\007.Action\"^\n\021ResultOrExcepti" + + "on\022\r\n\005index\030\001 \001(\r\022\027\n\006result\030\002 \001(\0132\007.Resu" + + "lt\022!\n\texception\030\003 \001(\0132\016.NameBytesPair\"C\n" + + "\022RegionActionResult\022-\n\021resultOrException" + + "\030\001 \003(\0132\022.ResultOrException\"3\n\014MultiReque" + + "st\022#\n\014regionAction\030\001 \003(\0132\r.RegionAction\"" + + "@\n\rMultiResponse\022/\n\022regionActionResult\030\001" + + " \003(\0132\023.RegionActionResult2\342\002\n\rClientServ" + + "ice\022 \n\003Get\022\013.GetRequest\032\014.GetResponse\022/\n" + + "\010MultiGet\022\020.MultiGetRequest\032\021.MultiGetRe", + "sponse\022)\n\006Mutate\022\016.MutateRequest\032\017.Mutat" + + "eResponse\022#\n\004Scan\022\014.ScanRequest\032\r.ScanRe" + + "sponse\022>\n\rBulkLoadHFile\022\025.BulkLoadHFileR" + + "equest\032\026.BulkLoadHFileResponse\022F\n\013ExecSe" + + "rvice\022\032.CoprocessorServiceRequest\032\033.Copr" + + "ocessorServiceResponse\022&\n\005Multi\022\r.MultiR" + + "equest\032\016.MultiResponseBB\n*org.apache.had" + + "oop.hbase.protobuf.generatedB\014ClientProt" + + "osH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -28027,30 +29777,42 @@ public final class ClientProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CoprocessorServiceResponse_descriptor, new java.lang.String[] { "Region", "Value", }); - internal_static_MultiAction_descriptor = + internal_static_Action_descriptor = getDescriptor().getMessageTypes().get(19); - internal_static_MultiAction_fieldAccessorTable = new + internal_static_Action_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_MultiAction_descriptor, - new java.lang.String[] { "Mutation", "Get", }); - internal_static_ActionResult_descriptor = + internal_static_Action_descriptor, + new java.lang.String[] { "Index", "Mutation", "Get", }); + internal_static_RegionAction_descriptor = getDescriptor().getMessageTypes().get(20); - internal_static_ActionResult_fieldAccessorTable = new + internal_static_RegionAction_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_ActionResult_descriptor, - new java.lang.String[] { "Value", "Exception", }); - internal_static_MultiRequest_descriptor = + internal_static_RegionAction_descriptor, + new java.lang.String[] { "Region", "Atomic", "Action", }); + internal_static_ResultOrException_descriptor = getDescriptor().getMessageTypes().get(21); + internal_static_ResultOrException_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ResultOrException_descriptor, + new java.lang.String[] { "Index", "Result", "Exception", }); + internal_static_RegionActionResult_descriptor = + getDescriptor().getMessageTypes().get(22); + internal_static_RegionActionResult_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RegionActionResult_descriptor, + new java.lang.String[] { "ResultOrException", }); + internal_static_MultiRequest_descriptor = + getDescriptor().getMessageTypes().get(23); internal_static_MultiRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiRequest_descriptor, - new java.lang.String[] { "Region", "Action", "Atomic", }); + new java.lang.String[] { "RegionAction", }); internal_static_MultiResponse_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(24); internal_static_MultiResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiResponse_descriptor, - new java.lang.String[] { "Result", }); + new java.lang.String[] { "RegionActionResult", }); return null; } }; diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java index 603f4c5..313d7b2 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java @@ -3662,6 +3662,26 @@ public final class RPCProtos { * */ org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder(); + + // optional uint32 priority = 6; + /** + * optional uint32 priority = 6; + * + *
+     * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+     * See HConstants.
+     * 
+ */ + boolean hasPriority(); + /** + * optional uint32 priority = 6; + * + *
+     * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+     * See HConstants.
+     * 
+ */ + int getPriority(); } /** * Protobuf type {@code RequestHeader} @@ -3759,6 +3779,11 @@ public final class RPCProtos { bitField0_ |= 0x00000010; break; } + case 48: { + bitField0_ |= 0x00000020; + priority_ = input.readUInt32(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -3946,12 +3971,39 @@ public final class RPCProtos { return cellBlockMeta_; } + // optional uint32 priority = 6; + public static final int PRIORITY_FIELD_NUMBER = 6; + private int priority_; + /** + * optional uint32 priority = 6; + * + *
+     * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+     * See HConstants.
+     * 
+ */ + public boolean hasPriority() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional uint32 priority = 6; + * + *
+     * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+     * See HConstants.
+     * 
+ */ + public int getPriority() { + return priority_; + } + private void initFields() { callId_ = 0; traceInfo_ = org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo.getDefaultInstance(); methodName_ = ""; requestParam_ = false; cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance(); + priority_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -3980,6 +4032,9 @@ public final class RPCProtos { if (((bitField0_ & 0x00000010) == 0x00000010)) { output.writeMessage(5, cellBlockMeta_); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeUInt32(6, priority_); + } getUnknownFields().writeTo(output); } @@ -4009,6 +4064,10 @@ public final class RPCProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(5, cellBlockMeta_); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(6, priority_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -4057,6 +4116,11 @@ public final class RPCProtos { result = result && getCellBlockMeta() .equals(other.getCellBlockMeta()); } + result = result && (hasPriority() == other.hasPriority()); + if (hasPriority()) { + result = result && (getPriority() + == other.getPriority()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -4090,6 +4154,10 @@ public final class RPCProtos { hash = (37 * hash) + CELL_BLOCK_META_FIELD_NUMBER; hash = (53 * hash) + getCellBlockMeta().hashCode(); } + if (hasPriority()) { + hash = (37 * hash) + PRIORITY_FIELD_NUMBER; + hash = (53 * hash) + getPriority(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -4223,6 +4291,8 @@ public final class RPCProtos { cellBlockMetaBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000010); + priority_ = 0; + bitField0_ = (bitField0_ & ~0x00000020); return this; } @@ -4279,6 +4349,10 @@ public final class RPCProtos { } else { result.cellBlockMeta_ = cellBlockMetaBuilder_.build(); } + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.priority_ = priority_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -4312,6 +4386,9 @@ public final class RPCProtos { if (other.hasCellBlockMeta()) { mergeCellBlockMeta(other.getCellBlockMeta()); } + if (other.hasPriority()) { + setPriority(other.getPriority()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -4781,6 +4858,59 @@ public final class RPCProtos { return cellBlockMetaBuilder_; } + // optional uint32 priority = 6; + private int priority_ ; + /** + * optional uint32 priority = 6; + * + *
+       * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+       * See HConstants.
+       * 
+ */ + public boolean hasPriority() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional uint32 priority = 6; + * + *
+       * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+       * See HConstants.
+       * 
+ */ + public int getPriority() { + return priority_; + } + /** + * optional uint32 priority = 6; + * + *
+       * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+       * See HConstants.
+       * 
+ */ + public Builder setPriority(int value) { + bitField0_ |= 0x00000020; + priority_ = value; + onChanged(); + return this; + } + /** + * optional uint32 priority = 6; + * + *
+       * 0 is NORMAL priority.  100 is HIGH.  If no priority, treat it as NORMAL.
+       * See HConstants.
+       * 
+ */ + public Builder clearPriority() { + bitField0_ = (bitField0_ & ~0x00000020); + priority_ = 0; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:RequestHeader) } @@ -5797,15 +5927,15 @@ public final class RPCProtos { "\001(\r\"|\n\021ExceptionResponse\022\034\n\024exception_cl" + "ass_name\030\001 \001(\t\022\023\n\013stack_trace\030\002 \001(\t\022\020\n\010h" + "ostname\030\003 \001(\t\022\014\n\004port\030\004 \001(\005\022\024\n\014do_not_re", - "try\030\005 \001(\010\"\224\001\n\rRequestHeader\022\017\n\007call_id\030\001" + + "try\030\005 \001(\010\"\246\001\n\rRequestHeader\022\017\n\007call_id\030\001" + " \001(\r\022\035\n\ntrace_info\030\002 \001(\0132\t.RPCTInfo\022\023\n\013m" + "ethod_name\030\003 \001(\t\022\025\n\rrequest_param\030\004 \001(\010\022" + "\'\n\017cell_block_meta\030\005 \001(\0132\016.CellBlockMeta" + - "\"q\n\016ResponseHeader\022\017\n\007call_id\030\001 \001(\r\022%\n\te" + - "xception\030\002 \001(\0132\022.ExceptionResponse\022\'\n\017ce" + - "ll_block_meta\030\003 \001(\0132\016.CellBlockMetaB<\n*o" + - "rg.apache.hadoop.hbase.protobuf.generate" + - "dB\tRPCProtosH\001\240\001\001" + "\022\020\n\010priority\030\006 \001(\r\"q\n\016ResponseHeader\022\017\n\007" + + "call_id\030\001 \001(\r\022%\n\texception\030\002 \001(\0132\022.Excep" + + "tionResponse\022\'\n\017cell_block_meta\030\003 \001(\0132\016." + + "CellBlockMetaB<\n*org.apache.hadoop.hbase" + + ".protobuf.generatedB\tRPCProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -5841,7 +5971,7 @@ public final class RPCProtos { internal_static_RequestHeader_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RequestHeader_descriptor, - new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", }); + new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", "Priority", }); internal_static_ResponseHeader_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_ResponseHeader_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index 3629504..3c62844 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -318,43 +318,54 @@ message CoprocessorServiceResponse { required NameBytesPair value = 2; } +// Either a Get or a Mutation +message Action { + // If part of a multi action, useful aligning + // result with what was originally submitted. + optional uint32 index = 1; + optional MutationProto mutation = 2; + optional Get get = 3; +} + /** - * An action that is part of MultiRequest. - * This is a union type - exactly one of the fields will be set. + * Actions to run against a Region. */ -message MultiAction { - optional MutationProto mutation = 1; - optional Get get = 2; +message RegionAction { + required RegionSpecifier region = 1; + // When set, run mutations as atomic unit. + optional bool atomic = 2; + repeated Action action = 3; } /** - * An individual action result. The result will in the - * same order as the action in the request. If an action - * returns a value, it is set in value field. If it doesn't - * return anything, the result will be empty. If an action - * fails to execute due to any exception, the exception - * is returned as a stringified parameter. + * Either a Result or an Exception NameBytesPair (keyed by + * exception name whose value is the exception stringified) + * or maybe empty if no result and no exception. */ -message ActionResult { - optional Result value = 1; - optional NameBytesPair exception = 2; +message ResultOrException { + // If part of a multi call, save original index of the list of all + // passed so can align this response w/ original request. + optional uint32 index = 1; + optional Result result = 2; + optional NameBytesPair exception = 3; } /** - * You can execute a list of actions on a given region in order. - * - * If it is a list of mutate actions, atomic can be set - * to make sure they can be processed atomically, just like - * RowMutations. + * The result of a RegionAction. + */ +message RegionActionResult { + repeated ResultOrException resultOrException = 1; +} + +/** + * Execute a list of actions on a given region in order. */ message MultiRequest { - required RegionSpecifier region = 1; - repeated MultiAction action = 2; - optional bool atomic = 3; + repeated RegionAction regionAction = 1; } message MultiResponse { - repeated ActionResult result = 1; + repeated RegionActionResult regionActionResult = 1; } diff --git a/hbase-protocol/src/main/protobuf/RPC.proto b/hbase-protocol/src/main/protobuf/RPC.proto index 78b09b6..9bf69a1 100644 --- a/hbase-protocol/src/main/protobuf/RPC.proto +++ b/hbase-protocol/src/main/protobuf/RPC.proto @@ -119,7 +119,9 @@ message RequestHeader { optional bool request_param = 4; // If present, then an encoded data block follows. optional CellBlockMeta cell_block_meta = 5; - // TODO: Have client specify priority + // 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL. + // See HConstants. + optional uint32 priority = 6; } message ResponseHeader { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java index aa7018b..6051769 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java @@ -83,8 +83,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { CompactRegionRequest.class, GetRequest.class, MutateRequest.class, - ScanRequest.class, - MultiRequest.class + ScanRequest.class }; // Some caches for helping performance @@ -101,7 +100,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { if (p != null) { // Since we protobuf'd, and then subsequently, when we went with pb style, method names // are capitalized. This meant that this brittle compare of method names gotten by - // reflection no longer matched the method names comeing in over pb. TODO: Get rid of this + // reflection no longer matched the method names coming in over pb. TODO: Get rid of this // check. For now, workaround is to capitalize the names we got from reflection so they // have chance of matching the pb ones. String capitalizedMethodName = capitalize(m.getName()); @@ -109,7 +108,6 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { } } this.annotatedQos = qosMap; - if (methodMap.get("getRegion") == null) { methodMap.put("hasRegion", new HashMap, Method>()); methodMap.put("getRegion", new HashMap, Method>()); @@ -148,10 +146,14 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { if (priorityByAnnotation != null) { return priorityByAnnotation; } - if (param == null) { return HConstants.NORMAL_QOS; } + if (methodName.equalsIgnoreCase("multi") && param instanceof MultiRequest) { + // The multi call has its priority set in the header. All calls should work this way but + // only this one has been converted so far. No priority == NORMAL_QOS. + return header.hasPriority()? header.getPriority(): HConstants.NORMAL_QOS; + } String cls = param.getClass().getName(); Class rpcArgClass = argumentToClassMap.get(cls); RegionSpecifier regionSpecifier = null; @@ -201,4 +203,4 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { void setRegionServer(final HRegionServer hrs) { this.hRegionServer = hrs; } -} \ No newline at end of file +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 93a78de..403b41d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -73,16 +73,16 @@ import org.apache.hadoop.hbase.DroppedSnapshotException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.NotServingRegionException; -import org.apache.hadoop.hbase.RegionTooBusyException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.RegionTooBusyException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; @@ -113,11 +113,9 @@ import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index fc53c9a..61e5313 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -18,8 +18,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import javax.management.ObjectName; - import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; import java.lang.annotation.Retention; @@ -38,7 +36,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; import java.util.Random; import java.util.Set; @@ -50,6 +47,8 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.management.ObjectName; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -64,8 +63,8 @@ import org.apache.hadoop.hbase.Chore; import org.apache.hadoop.hbase.ClockOutOfSyncException; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HealthCheckChore; @@ -90,7 +89,6 @@ import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -120,7 +118,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest; @@ -138,7 +135,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRespon import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState; @@ -152,8 +148,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry; 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.BulkLoadHFileRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse; @@ -170,6 +166,9 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos; @@ -185,7 +184,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey; import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; @@ -3305,108 +3303,130 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa // It is also the conduit via which we pass back data. PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc; CellScanner cellScanner = controller != null ? controller.cellScanner(): null; - // Clear scanner so we are not holding on to reference across call. if (controller != null) controller.setCellScanner(null); List cellsToReturn = null; + MultiResponse.Builder responseBuilder = MultiResponse.newBuilder(); try { - HRegion region = getRegion(request.getRegion()); - MultiResponse.Builder builder = MultiResponse.newBuilder(); - List mutations = new ArrayList(request.getActionCount()); - // Do a bunch of mutations atomically. Mutations are Puts and Deletes. NOT Gets. - if (request.hasAtomic() && request.getAtomic()) { - // MultiAction is union type. Has a Get or a Mutate. - for (ClientProtos.MultiAction actionUnion : request.getActionList()) { - if (actionUnion.hasMutation()) { - mutations.add(actionUnion.getMutation()); - } else { - throw new DoNotRetryIOException("Unsupported atomic action type: " + actionUnion); - } - } - // TODO: We are not updating a metric here. Should we up requestCount? - if (!mutations.isEmpty()) mutateRows(region, mutations, cellScanner); - } else { - // Do a bunch of Actions. - ActionResult.Builder resultBuilder = null; - cellsToReturn = new ArrayList(request.getActionCount()); - for (ClientProtos.MultiAction actionUnion : request.getActionList()) { + for (RegionAction regionAction: request.getRegionActionList()) { + RegionActionResult.Builder regionActionResultBuilder = null; + HRegion region = getRegion(regionAction.getRegion()); + if (regionAction.hasAtomic() && regionAction.getAtomic()) { this.requestCount.increment(); - ClientProtos.Result result = null; - try { - if (actionUnion.hasGet()) { - Get get = ProtobufUtil.toGet(actionUnion.getGet()); - Result r = region.get(get); - if (r != null) { - // Get a result with no data. The data will be carried alongside pbs, not as pbs. - result = ProtobufUtil.toResultNoData(r); - // Add the Result to controller so it gets serialized apart from pb. Get - // Results could be big so good if they are not serialized as pb. - cellsToReturn.add(r); - } - } else if (actionUnion.hasMutation()) { - MutationProto mutation = actionUnion.getMutation(); - MutationType type = mutation.getMutateType(); - if (type != MutationType.PUT && type != MutationType.DELETE) { - if (!mutations.isEmpty()) { - doBatchOp(builder, region, mutations, cellScanner); - mutations.clear(); - } else if (!region.getRegionInfo().isMetaTable()) { - cacheFlusher.reclaimMemStoreMemory(); - } - } - Result r = null; - switch (type) { - case APPEND: - r = append(region, mutation, cellScanner); - break; - case INCREMENT: - r = increment(region, mutation, cellScanner); - break; - case PUT: - case DELETE: - mutations.add(mutation); - break; - default: - throw new DoNotRetryIOException("Unsupported mutate type: " + type.name()); - } - if (r != null) { - // Put the data into the cellsToReturn and the metadata about the result is all that - // we will pass back in the protobuf result. - result = ProtobufUtil.toResultNoData(r); - cellsToReturn.add(r); - } - } else { - LOG.warn("Error: invalid action: " + actionUnion + ". " - + "it must be a Get, Mutate, or Exec."); - throw new DoNotRetryIOException("Invalid action, " - + "it must be a Get, Mutate, or Exec."); - } - if (result != null) { - if (resultBuilder == null) { - resultBuilder = ActionResult.newBuilder(); - } else { - resultBuilder.clear(); - } - resultBuilder.setValue(result); - builder.addResult(resultBuilder.build()); - } - } catch (IOException ie) { - builder.addResult(ResponseConverter.buildActionResult(ie)); - } - } - if (!mutations.isEmpty()) { - doBatchOp(builder, region, mutations, cellScanner); + // How does this call happen? It may need some work to play well w/ the surroundings. + // Need to return an item per Action along w/ Action index. TODO. + mutateRows(region, regionAction.getActionList(), cellScanner); + } else { + regionActionResultBuilder = RegionActionResult.newBuilder(); + cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner, + regionActionResultBuilder, cellsToReturn); } + // Have one regionmutationresult per regionmutation even if it is empty so we keep results + // aligned w/ how the requests came in. + responseBuilder.addRegionActionResult(regionActionResultBuilder == null? + RegionActionResult.getDefaultInstance(): regionActionResultBuilder.build()); } // Load the controller with the Cells to return. if (cellsToReturn != null && !cellsToReturn.isEmpty() && controller != null) { controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn)); } - return builder.build(); + return responseBuilder.build(); } catch (IOException ie) { throw new ServiceException(ie); } } + /** + * Run through the regionMutation rm and per Mutation, do the work, and then when + * done, add an instance of a {@link ResultOrException} that corresponds to each Mutation. + * @param region + * @param rm + * @param cellScanner + * @param builder + * @param cellsToReturn Could be null. May be allocated in this method. This is what this + * method returns as a 'result'. + * @return Return the cellScanner passed + */ + private List doNonAtomicRegionMutation(final HRegion region, + final RegionAction actions, final CellScanner cellScanner, + final RegionActionResult.Builder builder, List cellsToReturn) { + // Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do + // one at a time, we instead pass them in batch. Be aware that the corresponding + // ResultOrException instance that matches each Put or Delete is then added down in the + // doBatchOp call. We should be staying aligned though the Put and Delete are deferred/batched + List mutations = null; + for (ClientProtos.Action action: actions.getActionList()) { + ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null; + try { + Result r = null; + if (action.hasGet()) { + Get get = ProtobufUtil.toGet(action.getGet()); + LOG.info("SERVER GET " + get); + r = region.get(get); + LOG.info("SERVER RESULT " + r); + } else if (action.hasMutation()) { + MutationType type = action.getMutation().getMutateType(); + if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null && + !mutations.isEmpty()) { + // Flush out any Puts or Deletes already collected. + doBatchOp(builder, region, mutations, cellScanner); + mutations.clear(); + } + switch (type) { + case APPEND: + r = append(region, action.getMutation(), cellScanner); + break; + case INCREMENT: + r = increment(region, action.getMutation(), cellScanner); + break; + case PUT: + case DELETE: + // Collect the individual mutations and apply in a batch + if (mutations == null) { + mutations = new ArrayList(actions.getActionCount()); + } + mutations.add(action); + break; + default: + throw new DoNotRetryIOException("Unsupported mutate type: " + type.name()); + } + } else { + throw new HBaseIOException("Unexpected Action type"); + } + if (r != null) { + ClientProtos.Result pbResult = null; + if (isClientCellBlockSupport()) { + pbResult = ProtobufUtil.toResultNoData(r); + // Hard to guess the size here. Just make a rough guess. + if (cellsToReturn == null) cellsToReturn = new ArrayList(); + LOG.info("SERVER ADD RESULT " + r); + cellsToReturn.add(r); + } else { + pbResult = ProtobufUtil.toResult(r); + } + resultOrExceptionBuilder = + ClientProtos.ResultOrException.newBuilder().setResult(pbResult); + } + // Could get to here and there was no result and no exception. Presumes we added + // a Put or Delete to the collecting Mutations List for adding later. In this + // case the corresponding ResultOrException instance for the Put or Delete will be added + // down in the doBatchOp method call rather than up here. + } catch (IOException ie) { + resultOrExceptionBuilder = ResultOrException.newBuilder(). + setException(ResponseConverter.buildException(ie)); + } + if (resultOrExceptionBuilder != null) { + // Propagate index. + resultOrExceptionBuilder.setIndex(action.getIndex()); + builder.addResultOrException(resultOrExceptionBuilder.build()); + } + } + // Finish up any outstanding mutations + if (mutations != null && !mutations.isEmpty()) { + doBatchOp(builder, region, mutations, cellScanner); + } + return cellsToReturn; + } + // End Client methods // Start Admin methods @@ -3882,11 +3902,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa try { checkOpen(); List entries = request.getEntryList(); - if(entries == null || entries.isEmpty()) { + if (entries == null || entries.isEmpty()) { // empty input return ReplicateWALEntryResponse.newBuilder().build(); } - HRegion region = this.getRegionByEncodedName( entries.get(0).getKey().getEncodedRegionName().toStringUtf8()); RegionCoprocessorHost coprocessorHost = region.getCoprocessorHost(); @@ -4067,17 +4086,17 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * @param region * @param mutations */ - protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region, - final List mutations, final CellScanner cells) { + protected void doBatchOp(final RegionActionResult.Builder builder, final HRegion region, + final List mutations, final CellScanner cells) { Mutation[] mArray = new Mutation[mutations.size()]; long before = EnvironmentEdgeManager.currentTimeMillis(); boolean batchContainsPuts = false, batchContainsDelete = false; try { - ActionResult.Builder resultBuilder = ActionResult.newBuilder(); - resultBuilder.setValue(ClientProtos.Result.newBuilder().build()); - ActionResult result = resultBuilder.build(); int i = 0; - for (MutationProto m : mutations) { + for (ClientProtos.Action action: mutations) { + ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = + ClientProtos.ResultOrException.newBuilder(); + MutationProto m = action.getMutation(); Mutation mutation; if (m.getMutateType() == MutationType.PUT) { mutation = ProtobufUtil.toPut(m, cells); @@ -4087,7 +4106,6 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa batchContainsDelete = true; } mArray[i++] = mutation; - builder.addResult(result); } requestCount.add(mutations.size()); @@ -4097,33 +4115,33 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa OperationStatus codes[] = region.batchMutate(mArray, false); for (i = 0; i < codes.length; i++) { + int index = mutations.get(i).getIndex(); + Exception e = null; switch (codes[i].getOperationStatusCode()) { case BAD_FAMILY: - result = ResponseConverter.buildActionResult( - new NoSuchColumnFamilyException(codes[i].getExceptionMsg())); - builder.setResult(i, result); + e = new NoSuchColumnFamilyException(codes[i].getExceptionMsg()); + builder.addResultOrException(getResultOrException(e, index)); break; case SANITY_CHECK_FAILURE: - result = ResponseConverter.buildActionResult( - new FailedSanityCheckException(codes[i].getExceptionMsg())); - builder.setResult(i, result); + e = new FailedSanityCheckException(codes[i].getExceptionMsg()); + builder.addResultOrException(getResultOrException(e, index)); break; default: - result = ResponseConverter.buildActionResult( - new DoNotRetryIOException(codes[i].getExceptionMsg())); - builder.setResult(i, result); + e = new DoNotRetryIOException(codes[i].getExceptionMsg()); + builder.addResultOrException(getResultOrException(e, index)); break; case SUCCESS: + builder.addResultOrException(getResultOrException(ClientProtos.Result.getDefaultInstance(), index)); break; } } } catch (IOException ie) { - ActionResult result = ResponseConverter.buildActionResult(ie); + ResultOrException resultOrException = ResponseConverter.buildActionResult(ie).build(); for (int i = 0; i < mutations.size(); i++) { - builder.setResult(i, result); + builder.setResultOrException(i, resultOrException); } } long after = EnvironmentEdgeManager.currentTimeMillis(); @@ -4134,6 +4152,18 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa metricsRegionServer.updateDelete(after - before); } } + private static ResultOrException getResultOrException(final ClientProtos.Result r, + final int index) { + return getResultOrException(ResponseConverter.buildActionResult(r), index); + } + private static ResultOrException getResultOrException(final Exception e, final int index) { + return getResultOrException(ResponseConverter.buildActionResult(e), index); + } + + private static ResultOrException getResultOrException(final ResultOrException.Builder builder, + final int index) { + return builder.setIndex(index).build(); + } /** * Execute a list of Put/Delete mutations. The function returns OperationStatus instead of @@ -4145,8 +4175,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * exceptionMessage if any * @throws IOException */ - protected OperationStatus[] doBatchOp(final HRegion region, - final List> mutations, boolean isReplay) throws IOException { + protected OperationStatus [] doBatchOp(final HRegion region, + final List> mutations, boolean isReplay) + throws IOException { Mutation[] mArray = new Mutation[mutations.size()]; long before = EnvironmentEdgeManager.currentTimeMillis(); boolean batchContainsPuts = false, batchContainsDelete = false; @@ -4184,28 +4215,31 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * @param cellScanner if non-null, the mutation data -- the Cell content. * @throws IOException */ - protected void mutateRows(final HRegion region, final List mutations, + protected void mutateRows(final HRegion region, final List actions, final CellScanner cellScanner) throws IOException { - MutationProto firstMutate = mutations.get(0); if (!region.getRegionInfo().isMetaTable()) { cacheFlusher.reclaimMemStoreMemory(); } - byte [] row = firstMutate.getRow().toByteArray(); - RowMutations rm = new RowMutations(row); - for (MutationProto mutate: mutations) { - MutationType type = mutate.getMutateType(); - switch (mutate.getMutateType()) { + RowMutations rm = null; + for (ClientProtos.Action action: actions) { + if (action.hasGet()) { + throw new DoNotRetryIOException("Atomic put and/or delete only, not a Get=" + + action.getGet()); + } + MutationType type = action.getMutation().getMutateType(); + if (rm == null) { + rm = new RowMutations(action.getMutation().getRow().toByteArray()); + } + switch (type) { case PUT: - rm.add(ProtobufUtil.toPut(mutate, cellScanner)); + rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner)); break; case DELETE: - rm.add(ProtobufUtil.toDelete(mutate, cellScanner)); + rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner)); break; default: - throw new DoNotRetryIOException( - "mutate supports atomic put and/or delete, not " - + type.name()); + throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name()); } } region.mutateRow(rm); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java index 4368a17..ecf1ea6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java @@ -23,8 +23,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.NavigableMap; -import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; @@ -32,35 +30,23 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.client.Action; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.RegionServerCallable; -import org.apache.hadoop.hbase.client.Row; -import org.apache.hadoop.hbase.client.RpcRetryingCaller; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; -import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.protobuf.generated.WALProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse; -import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; -import com.google.protobuf.ByteString; import com.google.protobuf.ServiceException; /** @@ -118,7 +104,7 @@ public class WALEditsReplaySink { HRegionLocation loc = null; HLog.Entry entry = null; List regionEntries = null; - // Build the action list. + // Build the action list. for (int i = 0; i < batchSize; i++) { loc = entries.get(i).getFirst(); entry = entries.get(i).getSecond(); @@ -130,7 +116,7 @@ public class WALEditsReplaySink { } regionEntries.add(entry); } - + long startTime = EnvironmentEdgeManager.currentTimeMillis(); // replaying edits by region @@ -143,7 +129,7 @@ public class WALEditsReplaySink { for (; replayedActions < totalActions;) { curBatchSize = (totalActions > (MAX_BATCH_SIZE + replayedActions)) ? MAX_BATCH_SIZE : (totalActions - replayedActions); - replayEdits(loc, curRegion, allActions.subList(replayedActions, + replayEdits(loc, curRegion, allActions.subList(replayedActions, replayedActions + curBatchSize)); replayedActions += curBatchSize; } @@ -185,7 +171,7 @@ public class WALEditsReplaySink { } } } - + /** * Callable that handles the replay method call going against a single regionserver * @param @@ -202,7 +188,7 @@ public class WALEditsReplaySink { this.regionInfo = regionInfo; setLocation(regionLoc); } - + @Override public ReplicateWALEntryResponse call() throws IOException { try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 2f108e6..c96ef5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -43,7 +43,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -51,12 +50,12 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterBase; -import org.apache.hadoop.hbase.master.ClusterStatusPublisher; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -447,12 +446,12 @@ public class TestHCM { try { table.put(put3); Assert.fail("Unreachable point"); - }catch (RetriesExhaustedWithDetailsException e){ + } catch (RetriesExhaustedWithDetailsException e){ LOG.info("Put done, exception caught: " + e.getClass()); Assert.assertEquals(1, e.getNumExceptions()); Assert.assertArrayEquals(e.getRow(0).getRow(), ROW); } - Assert.assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW)); + Assert.assertNotNull("Cached connection is null", conn.getCachedLocation(TABLE_NAME, ROW)); Assert.assertEquals( "Previous server was "+curServer.getServerName().getHostAndPort(), destServerName.getPort(), conn.getCachedLocation(TABLE_NAME, ROW).getPort()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index eb98634..979c521 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -179,7 +179,6 @@ public class TestMultiParallel { for (Row get : gets) { singleRes.add(table.get((Get) get)); } - // Compare results Assert.assertEquals(singleRes.size(), multiRes.length); for (int i = 0; i < singleRes.size(); i++) { @@ -332,16 +331,20 @@ public class TestMultiParallel { validateSizeAndEmpty(results, KEYS.length); if (true) { - int liveRScount = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads() - .size(); + int liveRScount = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size(); assert liveRScount > 0; - JVMClusterUtil.RegionServerThread liveRS = UTIL.getMiniHBaseCluster() - .getLiveRegionServerThreads().get(0); - liveRS.getRegionServer().abort("Aborting for tests", - new Exception("testBatchWithPut")); - + JVMClusterUtil.RegionServerThread liveRS = + UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().get(0); + liveRS.getRegionServer().abort("Aborting for tests", new Exception("testBatchWithPut")); puts = constructPutRequests(); - results = table.batch(puts); + try { + results = table.batch(puts); + } catch (RetriesExhaustedWithDetailsException ree) { + LOG.info(ree.getExhaustiveDescription()); + throw ree; + } finally { + table.close(); + } validateSizeAndEmpty(results, KEYS.length); } @@ -597,6 +600,4 @@ public class TestMultiParallel { validateEmpty(result); } } - -} - +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java index 7818bd3..4748915 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java @@ -21,12 +21,9 @@ package org.apache.hadoop.hbase.ipc; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.internal.verification.VerificationModeFactory.times; @@ -36,12 +33,9 @@ import java.net.InetSocketAddress; import java.net.Socket; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; import javax.net.SocketFactory; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -64,6 +58,7 @@ import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponsePr import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -72,11 +67,12 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.mockito.Matchers; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.protobuf.BlockingService; import com.google.protobuf.Descriptors.MethodDescriptor; import com.google.protobuf.Message; @@ -321,7 +317,7 @@ public class TestIPC { for (int i = 0; i < cycles; i++) { List cells = new ArrayList(); // Message param = RequestConverter.buildMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm); - Message param = RequestConverter.buildNoDataMultiRequest( + ClientProtos.RegionAction.Builder builder = RequestConverter.buildNoDataRegionAction( HConstants.EMPTY_BYTE_ARRAY, rm, cells); CellScanner cellScanner = CellUtil.createCellScanner(cells); if (i % 1000 == 0) { @@ -331,7 +327,7 @@ public class TestIPC { // "Thread dump " + Thread.currentThread().getName()); } Pair response = - client.call(null, param, cellScanner, null, user, address, 0); + client.call(null, builder.build(), cellScanner, null, user, address, 0); /* int count = 0; while (p.getSecond().advance()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java index bc8ede7..873e3ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java @@ -20,8 +20,8 @@ import static org.junit.Assert.assertEquals; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; -import org.apache.hadoop.hbase.util.Pair; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -43,11 +43,19 @@ public class TestQosFunction { checkMethod("ReplicateWALEntry", HConstants.REPLICATION_QOS, qosFunction); // Set method name in pb style with the method name capitalized. checkMethod("OpenRegion", HConstants.HIGH_QOS, qosFunction); + // Check multi works. + checkMethod("Multi", HConstants.NORMAL_QOS, qosFunction, MultiRequest.getDefaultInstance()); } - private void checkMethod(final String methodName, final int expected, final AnnotationReadingPriorityFunction qosf) { + private void checkMethod(final String methodName, final int expected, + final AnnotationReadingPriorityFunction qosf) { + checkMethod(methodName, expected, qosf, null); + } + + private void checkMethod(final String methodName, final int expected, + final AnnotationReadingPriorityFunction qosf, final Message param) { RequestHeader.Builder builder = RequestHeader.newBuilder(); builder.setMethodName(methodName); - assertEquals(methodName, expected, qosf.getPriority(builder.build(), null)); + assertEquals(methodName, expected, qosf.getPriority(builder.build(), param)); } } \ No newline at end of file