Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (working copy) @@ -255,12 +255,14 @@ * @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 @@ * @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 @@ 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 @@ } 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,12 +589,11 @@ // 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>(); - for (List> actions : rsActions.actions.values()) { - for (Action action : actions) { + List> toReplay = new ArrayList>(initialActions.size()); + for (Map.Entry>> e : rsActions.actions.entrySet()) { + for (Action action : e.getValue()) { if (manageError(numAttempt, action.getOriginalIndex(), action.getAction(), true, t, location)) { toReplay.add(action); @@ -605,7 +603,7 @@ 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 @@ } } 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 @@ 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; } @@ -705,10 +702,11 @@ if (failureCount != 0) { // We have a failure but nothing to retry. We're done, it's a final failure.. LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount + - " ops on " + location.getServerName() + " NOT resubmitting." + location); + " 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()); } } } Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java (working copy) @@ -171,6 +171,7 @@ ScanResponse response = null; PayloadCarryingRpcController controller = new PayloadCarryingRpcController(); try { + controller.setPriority(getTableName()); response = getStub().scan(controller, request); return ResponseConverter.getResults(controller.cellScanner(), response); Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java (working copy) @@ -32,7 +32,6 @@ 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; @@ -72,6 +71,8 @@ private int storeOffset = 0; private Filter filter = null; private TimeRange tr = new TimeRange(); + private boolean checkExistenceOnly = false; + private boolean closestRowBefore = false; private Map> familyMap = new TreeMap>(Bytes.BYTES_COMPARATOR); @@ -87,6 +88,22 @@ this.row = row; } + public boolean isCheckExistenceOnly() { + return checkExistenceOnly; + } + + public void setCheckExistenceOnly(boolean checkExistenceOnly) { + this.checkExistenceOnly = checkExistenceOnly; + } + + public boolean isClosestRowBefore() { + return closestRowBefore; + } + + public void setClosestRowBefore(boolean closestRowBefore) { + this.closestRowBefore = closestRowBefore; + } + /** * Get all columns from the specified family. *

Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (working copy) @@ -643,6 +643,7 @@ .getServerName()); PayloadCarryingRpcController controller = new PayloadCarryingRpcController(); try { + controller.setPriority(tableName); ScanResponse response = server.scan(controller, request); values = ResponseConverter.getResults(controller.cellScanner(), response); } catch (ServiceException se) { Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (working copy) @@ -2164,8 +2164,7 @@ 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. @@ -2362,6 +2361,7 @@ } } + /* * Return the number of cached region for a table. It will only be called * from a unit test. Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (working copy) @@ -23,7 +23,6 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -61,11 +60,10 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest; -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; @@ -85,9 +83,6 @@ *

In case of reads, some fields used by a Scan are shared among all threads. * The HTable implementation can either not contract to be safe in case of a Get * - *

To access a table in a multi threaded environment, please consider - * using the {@link HTablePool} class to create your HTable instances. - * *

Instances of HTable passed the same {@link Configuration} instance will * share connections to servers out on the cluster and to the zookeeper ensemble * as well as caches of region locations. This is usually a *good* thing and it @@ -959,8 +954,13 @@ 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 +987,7 @@ 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 +1014,10 @@ 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 +1076,7 @@ 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()); @@ -1142,164 +1145,39 @@ */ @Override public boolean exists(final Get get) throws IOException { - RegionServerCallable callable = - new RegionServerCallable(connection, getName(), get.getRow()) { - public Boolean call() throws IOException { - try { - GetRequest request = RequestConverter.buildGetRequest( - getLocation().getRegionInfo().getRegionName(), get, true); - GetResponse response = getStub().get(null, request); - return response.getExists(); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - }; - return rpcCallerFactory. newCaller().callWithRetries(callable, this.operationTimeout); + get.setCheckExistenceOnly(true); + Result r = get(get); + assert r.getExists() != null; + return r.getExists(); } /** - * Goal of this inner class is to keep track of the initial position of a get in a list before - * sorting it. This is used to send back results in the same orders we got the Gets before we sort - * them. - */ - private static class SortedGet implements Comparable { - protected int initialIndex = -1; // Used to store the get initial index in a list. - protected Get get; // Encapsulated Get instance. - - public SortedGet (Get get, int initialIndex) { - this.get = get; - this.initialIndex = initialIndex; - } - - public int getInitialIndex() { - return initialIndex; - } - - @Override - public int compareTo(SortedGet o) { - return get.compareTo(o.get); - } - - public Get getGet() { - return get; - } - - @Override - public int hashCode() { - return get.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof SortedGet) - return get.equals(((SortedGet)obj).get); - else - return false; - } - } - - /** * {@inheritDoc} */ @Override public Boolean[] exists(final List gets) throws IOException { - // Prepare the sorted list of gets. Take the list of gets received, and encapsulate them into - // a list of SortedGet instances. Simple list parsing, so complexity here is O(n) - // The list is later used to recreate the response order based on the order the Gets - // got received. - ArrayList sortedGetsList = new ArrayList(); - for (int indexGet = 0; indexGet < gets.size(); indexGet++) { - sortedGetsList.add(new SortedGet (gets.get(indexGet), indexGet)); - } + if (gets.isEmpty()) return new Boolean[]{}; + if (gets.size() == 1) return new Boolean[]{exists(gets.get(0))}; - // Sorting the list to get the Gets ordered based on the key. - Collections.sort(sortedGetsList); // O(n log n) - - // step 1: sort the requests by regions to send them bundled. - // Map key is startKey index. Map value is the list of Gets related to the region starting - // with the startKey. - Map> getsByRegion = new HashMap>(); - - // Reference map to quickly find back in which region a get belongs. - Map getToRegionIndexMap = new HashMap(); - Pair startEndKeys = getStartEndKeys(); - - int regionIndex = 0; - for (final SortedGet get : sortedGetsList) { - // Progress on the regions until we find the one the current get resides in. - while ((regionIndex < startEndKeys.getSecond().length) && ((Bytes.compareTo(startEndKeys.getSecond()[regionIndex], get.getGet().getRow()) <= 0))) { - regionIndex++; - } - List regionGets = getsByRegion.get(regionIndex); - if (regionGets == null) { - regionGets = new ArrayList(); - getsByRegion.put(regionIndex, regionGets); - } - regionGets.add(get.getGet()); - getToRegionIndexMap.put(get.getGet(), regionIndex); + for (Get g: gets){ + g.setCheckExistenceOnly(true); } - // step 2: make the requests - Map>> futures = - new HashMap>>(sortedGetsList.size()); - for (final Map.Entry> getsByRegionEntry : getsByRegion.entrySet()) { - Callable> callable = new Callable>() { - public List call() throws Exception { - RegionServerCallable> callable = - new RegionServerCallable>(connection, getName(), - getsByRegionEntry.getValue().get(0).getRow()) { - public List call() throws IOException { - try { - MultiGetRequest requests = RequestConverter.buildMultiGetRequest( - getLocation().getRegionInfo().getRegionName(), getsByRegionEntry.getValue(), - true, false); - MultiGetResponse responses = getStub().multiGet(null, requests); - return responses.getExistsList(); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - }; - return rpcCallerFactory.> newCaller().callWithRetries(callable, - operationTimeout); - } - }; - futures.put(getsByRegionEntry.getKey(), pool.submit(callable)); + Object[] r1; + try { + r1 = batch(gets); + } catch (InterruptedException e) { + throw new IOException(e); } - // step 3: collect the failures and successes - Map> responses = new HashMap>(); - for (final Map.Entry> sortedGetEntry : getsByRegion.entrySet()) { - try { - Future> future = futures.get(sortedGetEntry.getKey()); - List resp = future.get(); - - if (resp == null) { - LOG.warn("Failed for gets on region: " + sortedGetEntry.getKey()); - } - responses.put(sortedGetEntry.getKey(), resp); - } catch (ExecutionException e) { - LOG.warn("Failed for gets on region: " + sortedGetEntry.getKey()); - } catch (InterruptedException e) { - LOG.warn("Failed for gets on region: " + sortedGetEntry.getKey()); - Thread.currentThread().interrupt(); - } + // translate. + Boolean[] results = new Boolean[r1.length]; + int i = 0; + for (Object o : r1) { + // batch ensures if there is a failure we get an exception instead + results[i++] = ((Result)o).getExists(); } - Boolean[] results = new Boolean[sortedGetsList.size()]; - // step 4: build the response. - Map indexes = new HashMap(); - for (int i = 0; i < sortedGetsList.size(); i++) { - Integer regionInfoIndex = getToRegionIndexMap.get(sortedGetsList.get(i).getGet()); - Integer index = indexes.get(regionInfoIndex); - if (index == null) { - index = 0; - } - results[sortedGetsList.get(i).getInitialIndex()] = responses.get(regionInfoIndex).get(index); - indexes.put(regionInfoIndex, index + 1); - } - return results; } Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (working copy) @@ -24,14 +24,17 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellScannable; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; 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,92 +45,75 @@ * @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++; - } + int countOfActions = this.multiAction.size(); + if (countOfActions <= 0) throw new DoNotRetryIOException("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; + 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); } - // 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)); - } + multiRequestBuilder.addRegionAction(regionActionBuilder.build()); + } + // 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 response; + return massFailedResponse; } - /** * @return True if we should send data in cellblocks. This is an expensive call. Cache the * result if you can rather than call each time. Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java (working copy) @@ -73,6 +73,7 @@ @InterfaceStability.Stable public class Result implements CellScannable { private Cell[] cells; + private Boolean exists; // if the query was just to check existence. // We're not using java serialization. Transient here is just a marker to say // that this is where we cache row if we're ever asked for it. private transient byte [] row = null; @@ -108,7 +109,7 @@ @Deprecated public Result(List kvs) { // TODO: Here we presume the passed in Cells are KVs. One day this won't always be so. - this(kvs.toArray(new Cell[kvs.size()])); + this(kvs.toArray(new Cell[kvs.size()]), null); } /** @@ -117,21 +118,29 @@ * @param cells List of cells */ public static Result create(List cells) { - return new Result(cells.toArray(new Cell[cells.size()])); + return new Result(cells.toArray(new Cell[cells.size()]), null); } + public static Result create(List cells, Boolean exists) { + if (exists != null){ + return new Result(null, exists); + } + return new Result(cells.toArray(new Cell[cells.size()]), exists); + } + /** * Instantiate a Result with the specified array of KeyValues. *
Note: You must ensure that the keyvalues are already sorted. * @param cells array of cells */ public static Result create(Cell[] cells) { - return new Result(cells); + return new Result(cells, null); } /** Private ctor. Use {@link #create(Cell[])}. */ - private Result(Cell[] cells) { + private Result(Cell[] cells, Boolean exists) { this.cells = cells; + this.exists = exists; } /** @@ -796,4 +805,12 @@ public CellScanner cellScanner() { return CellUtil.createCellScanner(this.cells); } + + public Boolean getExists() { + return exists; + } + + public void setExists(Boolean exists) { + this.exists = exists; + } } Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (working copy) @@ -163,6 +163,7 @@ 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 Index: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java (working copy) @@ -23,8 +23,9 @@ 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 @@ */ @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 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 Index: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java (working copy) @@ -55,6 +55,7 @@ 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 @@ @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 @@ } // 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 @@ * 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 @@ 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 @@ } } + 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 @@ */ 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 @@ } 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()); Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (working copy) @@ -406,6 +406,12 @@ } } } + if (proto.hasExistenceOnly() && proto.getExistenceOnly()){ + get.setCheckExistenceOnly(true); + } + if (proto.hasClosestRowBefore() && proto.getClosestRowBefore()){ + get.setClosestRowBefore(true); + } return get; } @@ -910,6 +916,12 @@ if (get.getRowOffsetPerColumnFamily() > 0) { builder.setStoreOffset(get.getRowOffsetPerColumnFamily()); } + if (get.isCheckExistenceOnly()){ + builder.setExistenceOnly(true); + } + if (get.isClosestRowBefore()){ + builder.setClosestRowBefore(true); + } return builder.build(); } @@ -1038,10 +1050,25 @@ builder.addCell(toCell(c)); } } + if (result.getExists() != null){ + builder.setExists(result.getExists()); + } return builder.build(); } /** + * Convert a client Result to a protocol buffer Result + * + * @param existence the client existence to send + * @return the converted protocol buffer Result + */ + public static ClientProtos.Result toResult(final boolean existence) { + ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder(); + builder.setExists(existence); + return builder.build(); + } + + /** * Convert a client Result to a protocol buffer Result. * The pb Result does not include the Cell data. That is for transport otherwise. * @@ -1051,6 +1078,9 @@ public static ClientProtos.Result toResultNoData(final Result result) { ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder(); builder.setAssociatedCellCount(result.size()); + if (result.getExists() != null){ + builder.setExists(result.getExists()); + } return builder.build(); } @@ -1061,12 +1091,16 @@ * @return the converted client Result */ public static Result toResult(final ClientProtos.Result proto) { + if (proto.hasExists()) { + return Result.create(null, proto.getExists()); + } + List values = proto.getCellList(); List cells = new ArrayList(values.size()); - for (CellProtos.Cell c: values) { + for (CellProtos.Cell c : values) { cells.add(toCell(c)); } - return Result.create(cells); + return Result.create(cells, null); } /** @@ -1079,6 +1113,10 @@ */ public static Result toResult(final ClientProtos.Result proto, final CellScanner scanner) throws IOException { + if (proto.hasExists()){ + return Result.create(null, proto.getExists()); + } + // TODO: Unit test that has some Cells in scanner and some in the proto. List cells = null; if (proto.hasAssociatedCellCount()) { @@ -1094,7 +1132,7 @@ for (CellProtos.Cell c: values) { cells.add(toCell(c)); } - return Result.create(cells); + return Result.create(cells, null); } /** @@ -2242,11 +2280,15 @@ ", 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()) + Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (working copy) @@ -63,14 +63,12 @@ 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 +99,7 @@ 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; @@ -131,7 +130,6 @@ GetRequest.Builder builder = GetRequest.newBuilder(); RegionSpecifier region = buildRegionSpecifier( RegionSpecifierType.REGION_NAME, regionName); - builder.setClosestRowBefore(true); builder.setRegion(region); Column.Builder columnBuilder = Column.newBuilder(); @@ -140,63 +138,30 @@ ClientProtos.Get.newBuilder(); getBuilder.setRow(ByteString.copyFrom(row)); getBuilder.addColumn(columnBuilder.build()); + getBuilder.setClosestRowBefore(true); builder.setGet(getBuilder.build()); return builder.build(); } - /** - * Create a protocol buffer GetRequest for a client Get - * - * @param regionName the name of the region to get - * @param get the client Get - * @return a protocol buffer GetReuqest - */ - public static GetRequest buildGetRequest(final byte[] regionName, - final Get get) throws IOException { - return buildGetRequest(regionName, get, false); - } /** * Create a protocol buffer GetRequest for a client Get * * @param regionName the name of the region to get * @param get the client Get - * @param existenceOnly indicate if check row existence only * @return a protocol buffer GetRequest */ public static GetRequest buildGetRequest(final byte[] regionName, - final Get get, final boolean existenceOnly) throws IOException { + final Get get) throws IOException { GetRequest.Builder builder = GetRequest.newBuilder(); RegionSpecifier region = buildRegionSpecifier( RegionSpecifierType.REGION_NAME, regionName); - builder.setExistenceOnly(existenceOnly); builder.setRegion(region); builder.setGet(ProtobufUtil.toGet(get)); return builder.build(); } /** - * Create a protocol buffer MultiGetRequest for client Gets All gets are going to be run against - * the same region. - * @param regionName the name of the region to get from - * @param gets the client Gets - * @param existenceOnly indicate if check rows existence only - * @return a protocol buffer MultiGetRequest - */ - public static MultiGetRequest buildMultiGetRequest(final byte[] regionName, final List gets, - final boolean existenceOnly, final boolean closestRowBefore) throws IOException { - MultiGetRequest.Builder builder = MultiGetRequest.newBuilder(); - RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName); - builder.setExistenceOnly(existenceOnly); - builder.setClosestRowBefore(closestRowBefore); - builder.setRegion(region); - for (Get get : gets) { - builder.addGet(ProtobufUtil.toGet(get)); - } - return builder.build(); - } - - /** * Create a protocol buffer MutateRequest for a client increment * * @param regionName @@ -358,17 +323,18 @@ } /** - * 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 +346,26 @@ 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 +378,16 @@ } 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 +476,43 @@ /** * 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 + throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow"); } 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 +523,7 @@ * 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 +531,22 @@ * @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 +557,29 @@ // 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 Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java =================================================================== --- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java (revision 1529348) +++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java (working copy) @@ -39,14 +39,19 @@ 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; @@ -68,27 +73,58 @@ /** * Get the results from a protocol buffer MultiResponse * - * @param proto the protocol buffer MultiResponse to convert + * @param request the protocol buffer MultiResponse to convert * @param cells Cells to go with the passed in proto. Can be null. * @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 IllegalStateException("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(); + + if (actionResult.hasException()){ + Throwable regionException = ProtobufUtil.toException(actionResult.getException()); + for (ClientProtos.Action a : actions.getActionList()){ + results.add(regionName, new Pair(a.getIndex(), regionException)); + } + continue; } + + if (actions.getActionCount() != actionResult.getResultOrExceptionCount()) { + throw new IllegalStateException("actions.getActionCount=" + actions.getActionCount() + + ", actionResult.getResultOrExceptionCount=" + + actionResult.getResultOrExceptionCount() + " for region " + actions.getRegion()); + } + + for (ResultOrException roe : actionResult.getResultOrExceptionList()) { + 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 { + // no result & no exception. Unexpected. + throw new IllegalStateException("No result & no exception roe=" + roe + + " for region " + actions.getRegion()); + } + } } + return results; } @@ -96,16 +132,36 @@ * Wrap a throwable to an action result. * * @param t - * @return an action result + * @return an action result builder */ - public static ActionResult buildActionResult(final Throwable t) { - ActionResult.Builder builder = ActionResult.newBuilder(); + 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 r + * @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 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(); } /** Index: hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java =================================================================== --- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java (revision 1529348) +++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java (working copy) @@ -172,7 +172,7 @@ */ static class MyConnectionImpl2 extends MyConnectionImpl { List hrl; - boolean usedRegions[]; + final boolean usedRegions[]; protected MyConnectionImpl2(List hrl) { super(c); @@ -186,7 +186,7 @@ int i = 0; for (HRegionLocation hr:hrl){ if (Arrays.equals(row, hr.getRegionInfo().getStartKey())){ - usedRegions[i] = true; + usedRegions[i] = true; return hr; } i++; @@ -475,9 +475,9 @@ private class MyCB implements AsyncProcess.AsyncProcessCallback { - private AtomicInteger successCalled = new AtomicInteger(0); - private AtomicInteger failureCalled = new AtomicInteger(0); - private AtomicInteger retriableFailure = new AtomicInteger(0); + private final AtomicInteger successCalled = new AtomicInteger(0); + private final AtomicInteger failureCalled = new AtomicInteger(0); + private final AtomicInteger retriableFailure = new AtomicInteger(0); @Override @@ -705,7 +705,7 @@ */ @Test public void testThreadCreation() throws Exception { - final int NB_REGS = 10000; + final int NB_REGS = 100; List hrls = new ArrayList(NB_REGS); List gets = new ArrayList(NB_REGS); for (int i = 0; i < NB_REGS; i++) { @@ -721,11 +721,13 @@ HTable ht = new HTable(); MyConnectionImpl2 con = new MyConnectionImpl2(hrls); ht.connection = con; - ht.batch(gets); + ht.batch(gets); + + Assert.assertEquals(con.ap.nbActions.get(), NB_REGS); - Assert.assertEquals(con.ap.nbMultiResponse.get(), 2); // 1 multi response per server - Assert.assertEquals(con.nbThreads.get(), 2); // 1 thread per server + Assert.assertEquals("1 multi response per server", 2, con.ap.nbMultiResponse.get()); + Assert.assertEquals("1 thread per server", 2, con.nbThreads.get()); int nbReg = 0; for (int i =0; ioptional uint32 store_offset = 9; */ int getStoreOffset(); + + // optional bool existence_only = 10 [default = false]; + /** + * optional bool existence_only = 10 [default = false]; + * + *
+     * The result isn't asked for, just check for
+     * the existence.
+     * 
+ */ + boolean hasExistenceOnly(); + /** + * optional bool existence_only = 10 [default = false]; + * + *
+     * The result isn't asked for, just check for
+     * the existence.
+     * 
+ */ + boolean getExistenceOnly(); + + // optional bool closest_row_before = 11 [default = false]; + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+     * If the row to get doesn't exist, return the
+     * closest row before.
+     * 
+ */ + boolean hasClosestRowBefore(); + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+     * If the row to get doesn't exist, return the
+     * closest row before.
+     * 
+ */ + boolean getClosestRowBefore(); } /** * Protobuf type {@code Get} * *
    **
-   * The protocol buffer version of Get
+   * The protocol buffer version of Get.
+   * Unless existence_only is specified, return all the requested data
+   * for the row that matches exactly, or the one that immediately
+   * precedes it if closest_row_before is specified.
    * 
*/ public static final class Get extends @@ -873,6 +916,16 @@ storeOffset_ = input.readUInt32(); break; } + case 80: { + bitField0_ |= 0x00000080; + existenceOnly_ = input.readBool(); + break; + } + case 88: { + bitField0_ |= 0x00000100; + closestRowBefore_ = input.readBool(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -1115,6 +1168,58 @@ return storeOffset_; } + // optional bool existence_only = 10 [default = false]; + public static final int EXISTENCE_ONLY_FIELD_NUMBER = 10; + private boolean existenceOnly_; + /** + * optional bool existence_only = 10 [default = false]; + * + *
+     * The result isn't asked for, just check for
+     * the existence.
+     * 
+ */ + public boolean hasExistenceOnly() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + /** + * optional bool existence_only = 10 [default = false]; + * + *
+     * The result isn't asked for, just check for
+     * the existence.
+     * 
+ */ + public boolean getExistenceOnly() { + return existenceOnly_; + } + + // optional bool closest_row_before = 11 [default = false]; + public static final int CLOSEST_ROW_BEFORE_FIELD_NUMBER = 11; + private boolean closestRowBefore_; + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+     * If the row to get doesn't exist, return the
+     * closest row before.
+     * 
+ */ + public boolean hasClosestRowBefore() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+     * If the row to get doesn't exist, return the
+     * closest row before.
+     * 
+ */ + public boolean getClosestRowBefore() { + return closestRowBefore_; + } + private void initFields() { row_ = com.google.protobuf.ByteString.EMPTY; column_ = java.util.Collections.emptyList(); @@ -1125,6 +1230,8 @@ cacheBlocks_ = true; storeLimit_ = 0; storeOffset_ = 0; + existenceOnly_ = false; + closestRowBefore_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -1187,6 +1294,12 @@ if (((bitField0_ & 0x00000040) == 0x00000040)) { output.writeUInt32(9, storeOffset_); } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeBool(10, existenceOnly_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeBool(11, closestRowBefore_); + } getUnknownFields().writeTo(output); } @@ -1232,6 +1345,14 @@ size += com.google.protobuf.CodedOutputStream .computeUInt32Size(9, storeOffset_); } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(10, existenceOnly_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(11, closestRowBefore_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1294,6 +1415,16 @@ result = result && (getStoreOffset() == other.getStoreOffset()); } + result = result && (hasExistenceOnly() == other.hasExistenceOnly()); + if (hasExistenceOnly()) { + result = result && (getExistenceOnly() + == other.getExistenceOnly()); + } + result = result && (hasClosestRowBefore() == other.hasClosestRowBefore()); + if (hasClosestRowBefore()) { + result = result && (getClosestRowBefore() + == other.getClosestRowBefore()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1343,6 +1474,14 @@ hash = (37 * hash) + STORE_OFFSET_FIELD_NUMBER; hash = (53 * hash) + getStoreOffset(); } + if (hasExistenceOnly()) { + hash = (37 * hash) + EXISTENCE_ONLY_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getExistenceOnly()); + } + if (hasClosestRowBefore()) { + hash = (37 * hash) + CLOSEST_ROW_BEFORE_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getClosestRowBefore()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -1419,7 +1558,10 @@ * *
      **
-     * The protocol buffer version of Get
+     * The protocol buffer version of Get.
+     * Unless existence_only is specified, return all the requested data
+     * for the row that matches exactly, or the one that immediately
+     * precedes it if closest_row_before is specified.
      * 
*/ public static final class Builder extends @@ -1495,6 +1637,10 @@ bitField0_ = (bitField0_ & ~0x00000080); storeOffset_ = 0; bitField0_ = (bitField0_ & ~0x00000100); + existenceOnly_ = false; + bitField0_ = (bitField0_ & ~0x00000200); + closestRowBefore_ = false; + bitField0_ = (bitField0_ & ~0x00000400); return this; } @@ -1577,6 +1723,14 @@ to_bitField0_ |= 0x00000040; } result.storeOffset_ = storeOffset_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000080; + } + result.existenceOnly_ = existenceOnly_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000100; + } + result.closestRowBefore_ = closestRowBefore_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1666,6 +1820,12 @@ if (other.hasStoreOffset()) { setStoreOffset(other.getStoreOffset()); } + if (other.hasExistenceOnly()) { + setExistenceOnly(other.getExistenceOnly()); + } + if (other.hasClosestRowBefore()) { + setClosestRowBefore(other.getClosestRowBefore()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -2597,6 +2757,112 @@ return this; } + // optional bool existence_only = 10 [default = false]; + private boolean existenceOnly_ ; + /** + * optional bool existence_only = 10 [default = false]; + * + *
+       * The result isn't asked for, just check for
+       * the existence.
+       * 
+ */ + public boolean hasExistenceOnly() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + /** + * optional bool existence_only = 10 [default = false]; + * + *
+       * The result isn't asked for, just check for
+       * the existence.
+       * 
+ */ + public boolean getExistenceOnly() { + return existenceOnly_; + } + /** + * optional bool existence_only = 10 [default = false]; + * + *
+       * The result isn't asked for, just check for
+       * the existence.
+       * 
+ */ + public Builder setExistenceOnly(boolean value) { + bitField0_ |= 0x00000200; + existenceOnly_ = value; + onChanged(); + return this; + } + /** + * optional bool existence_only = 10 [default = false]; + * + *
+       * The result isn't asked for, just check for
+       * the existence.
+       * 
+ */ + public Builder clearExistenceOnly() { + bitField0_ = (bitField0_ & ~0x00000200); + existenceOnly_ = false; + onChanged(); + return this; + } + + // optional bool closest_row_before = 11 [default = false]; + private boolean closestRowBefore_ ; + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+       * If the row to get doesn't exist, return the
+       * closest row before.
+       * 
+ */ + public boolean hasClosestRowBefore() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+       * If the row to get doesn't exist, return the
+       * closest row before.
+       * 
+ */ + public boolean getClosestRowBefore() { + return closestRowBefore_; + } + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+       * If the row to get doesn't exist, return the
+       * closest row before.
+       * 
+ */ + public Builder setClosestRowBefore(boolean value) { + bitField0_ |= 0x00000400; + closestRowBefore_ = value; + onChanged(); + return this; + } + /** + * optional bool closest_row_before = 11 [default = false]; + * + *
+       * If the row to get doesn't exist, return the
+       * closest row before.
+       * 
+ */ + public Builder clearClosestRowBefore() { + bitField0_ = (bitField0_ & ~0x00000400); + closestRowBefore_ = false; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:Get) } @@ -2688,6 +2954,26 @@ * */ int getAssociatedCellCount(); + + // optional bool exists = 3; + /** + * optional bool exists = 3; + * + *
+     * used for Get to check existence only. Not set if existence_only was not set to true
+     *  in the query.
+     * 
+ */ + boolean hasExists(); + /** + * optional bool exists = 3; + * + *
+     * used for Get to check existence only. Not set if existence_only was not set to true
+     *  in the query.
+     * 
+ */ + boolean getExists(); } /** * Protobuf type {@code Result} @@ -2753,6 +3039,11 @@ associatedCellCount_ = input.readInt32(); break; } + case 24: { + bitField0_ |= 0x00000002; + exists_ = input.readBool(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -2891,9 +3182,36 @@ return associatedCellCount_; } + // optional bool exists = 3; + public static final int EXISTS_FIELD_NUMBER = 3; + private boolean exists_; + /** + * optional bool exists = 3; + * + *
+     * used for Get to check existence only. Not set if existence_only was not set to true
+     *  in the query.
+     * 
+ */ + public boolean hasExists() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bool exists = 3; + * + *
+     * used for Get to check existence only. Not set if existence_only was not set to true
+     *  in the query.
+     * 
+ */ + public boolean getExists() { + return exists_; + } + private void initFields() { cell_ = java.util.Collections.emptyList(); associatedCellCount_ = 0; + exists_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2913,6 +3231,9 @@ if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeInt32(2, associatedCellCount_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(3, exists_); + } getUnknownFields().writeTo(output); } @@ -2930,6 +3251,10 @@ size += com.google.protobuf.CodedOutputStream .computeInt32Size(2, associatedCellCount_); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(3, exists_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -2960,6 +3285,11 @@ result = result && (getAssociatedCellCount() == other.getAssociatedCellCount()); } + result = result && (hasExists() == other.hasExists()); + if (hasExists()) { + result = result && (getExists() + == other.getExists()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -2981,6 +3311,10 @@ hash = (37 * hash) + ASSOCIATED_CELL_COUNT_FIELD_NUMBER; hash = (53 * hash) + getAssociatedCellCount(); } + if (hasExists()) { + hash = (37 * hash) + EXISTS_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getExists()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -3099,6 +3433,8 @@ } associatedCellCount_ = 0; bitField0_ = (bitField0_ & ~0x00000002); + exists_ = false; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -3140,6 +3476,10 @@ to_bitField0_ |= 0x00000001; } result.associatedCellCount_ = associatedCellCount_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000002; + } + result.exists_ = exists_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -3185,6 +3525,9 @@ if (other.hasAssociatedCellCount()) { setAssociatedCellCount(other.getAssociatedCellCount()); } + if (other.hasExists()) { + setExists(other.getExists()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -3611,6 +3954,59 @@ return this; } + // optional bool exists = 3; + private boolean exists_ ; + /** + * optional bool exists = 3; + * + *
+       * used for Get to check existence only. Not set if existence_only was not set to true
+       *  in the query.
+       * 
+ */ + public boolean hasExists() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bool exists = 3; + * + *
+       * used for Get to check existence only. Not set if existence_only was not set to true
+       *  in the query.
+       * 
+ */ + public boolean getExists() { + return exists_; + } + /** + * optional bool exists = 3; + * + *
+       * used for Get to check existence only. Not set if existence_only was not set to true
+       *  in the query.
+       * 
+ */ + public Builder setExists(boolean value) { + bitField0_ |= 0x00000004; + exists_ = value; + onChanged(); + return this; + } + /** + * optional bool exists = 3; + * + *
+       * used for Get to check existence only. Not set if existence_only was not set to true
+       *  in the query.
+       * 
+ */ + public Builder clearExists() { + bitField0_ = (bitField0_ & ~0x00000004); + exists_ = false; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:Result) } @@ -3652,48 +4048,6 @@ * required .Get get = 2; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder(); - - // optional bool closest_row_before = 3; - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - boolean hasClosestRowBefore(); - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - boolean getClosestRowBefore(); - - // optional bool existence_only = 4; - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - boolean hasExistenceOnly(); - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - boolean getExistenceOnly(); } /** * Protobuf type {@code GetRequest} @@ -3701,11 +4055,6 @@ *
    **
    * The get request. Perform a single Get operation.
-   * Unless existence_only is specified, return all the requested data
-   * for the row that matches exactly, or the one that immediately
-   * precedes it if closest_row_before is specified.
-   *
-   * If existence_only is set, only the existence will be returned.
    * 
*/ public static final class GetRequest extends @@ -3782,16 +4131,6 @@ bitField0_ |= 0x00000002; break; } - case 24: { - bitField0_ |= 0x00000004; - closestRowBefore_ = input.readBool(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - existenceOnly_ = input.readBool(); - break; - } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -3876,65 +4215,9 @@ return get_; } - // optional bool closest_row_before = 3; - public static final int CLOSEST_ROW_BEFORE_FIELD_NUMBER = 3; - private boolean closestRowBefore_; - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - public boolean hasClosestRowBefore() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - public boolean getClosestRowBefore() { - return closestRowBefore_; - } - - // optional bool existence_only = 4; - public static final int EXISTENCE_ONLY_FIELD_NUMBER = 4; - private boolean existenceOnly_; - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - public boolean hasExistenceOnly() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - public boolean getExistenceOnly() { - return existenceOnly_; - } - private void initFields() { region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance(); - closestRowBefore_ = false; - existenceOnly_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -3970,12 +4253,6 @@ if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeMessage(2, get_); } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBool(3, closestRowBefore_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBool(4, existenceOnly_); - } getUnknownFields().writeTo(output); } @@ -3993,14 +4270,6 @@ size += com.google.protobuf.CodedOutputStream .computeMessageSize(2, get_); } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(3, closestRowBefore_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(4, existenceOnly_); - } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -4034,16 +4303,6 @@ result = result && getGet() .equals(other.getGet()); } - result = result && (hasClosestRowBefore() == other.hasClosestRowBefore()); - if (hasClosestRowBefore()) { - result = result && (getClosestRowBefore() - == other.getClosestRowBefore()); - } - result = result && (hasExistenceOnly() == other.hasExistenceOnly()); - if (hasExistenceOnly()) { - result = result && (getExistenceOnly() - == other.getExistenceOnly()); - } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -4065,14 +4324,6 @@ hash = (37 * hash) + GET_FIELD_NUMBER; hash = (53 * hash) + getGet().hashCode(); } - if (hasClosestRowBefore()) { - hash = (37 * hash) + CLOSEST_ROW_BEFORE_FIELD_NUMBER; - hash = (53 * hash) + hashBoolean(getClosestRowBefore()); - } - if (hasExistenceOnly()) { - hash = (37 * hash) + EXISTENCE_ONLY_FIELD_NUMBER; - hash = (53 * hash) + hashBoolean(getExistenceOnly()); - } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -4150,11 +4401,6 @@ *
      **
      * The get request. Perform a single Get operation.
-     * Unless existence_only is specified, return all the requested data
-     * for the row that matches exactly, or the one that immediately
-     * precedes it if closest_row_before is specified.
-     *
-     * If existence_only is set, only the existence will be returned.
      * 
*/ public static final class Builder extends @@ -4206,10 +4452,6 @@ getBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); - closestRowBefore_ = false; - bitField0_ = (bitField0_ & ~0x00000004); - existenceOnly_ = false; - bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -4254,14 +4496,6 @@ } else { result.get_ = getBuilder_.build(); } - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.closestRowBefore_ = closestRowBefore_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.existenceOnly_ = existenceOnly_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -4284,12 +4518,6 @@ if (other.hasGet()) { mergeGet(other.getGet()); } - if (other.hasClosestRowBefore()) { - setClosestRowBefore(other.getClosestRowBefore()); - } - if (other.hasExistenceOnly()) { - setExistenceOnly(other.getExistenceOnly()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -4567,116 +4795,6 @@ return getBuilder_; } - // optional bool closest_row_before = 3; - private boolean closestRowBefore_ ; - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public boolean hasClosestRowBefore() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public boolean getClosestRowBefore() { - return closestRowBefore_; - } - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public Builder setClosestRowBefore(boolean value) { - bitField0_ |= 0x00000004; - closestRowBefore_ = value; - onChanged(); - return this; - } - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public Builder clearClosestRowBefore() { - bitField0_ = (bitField0_ & ~0x00000004); - closestRowBefore_ = false; - onChanged(); - return this; - } - - // optional bool existence_only = 4; - private boolean existenceOnly_ ; - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public boolean hasExistenceOnly() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public boolean getExistenceOnly() { - return existenceOnly_; - } - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public Builder setExistenceOnly(boolean value) { - bitField0_ |= 0x00000008; - existenceOnly_ = value; - onChanged(); - return this; - } - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public Builder clearExistenceOnly() { - bitField0_ = (bitField0_ & ~0x00000008); - existenceOnly_ = false; - onChanged(); - return this; - } - // @@protoc_insertion_point(builder_scope:GetRequest) } @@ -4688,1215 +4806,6 @@ // @@protoc_insertion_point(class_scope:GetRequest) } - public interface MultiGetRequestOrBuilder - 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 .Get get = 2; - /** - * repeated .Get get = 2; - */ - java.util.List - getGetList(); - /** - * repeated .Get get = 2; - */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get getGet(int index); - /** - * repeated .Get get = 2; - */ - int getGetCount(); - /** - * repeated .Get get = 2; - */ - java.util.List - getGetOrBuilderList(); - /** - * repeated .Get get = 2; - */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder( - int index); - - // optional bool closest_row_before = 3; - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - boolean hasClosestRowBefore(); - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - boolean getClosestRowBefore(); - - // optional bool existence_only = 4; - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - boolean hasExistenceOnly(); - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - boolean getExistenceOnly(); - } - /** - * Protobuf type {@code MultiGetRequest} - */ - public static final class MultiGetRequest extends - com.google.protobuf.GeneratedMessage - implements MultiGetRequestOrBuilder { - // Use MultiGetRequest.newBuilder() to construct. - private MultiGetRequest(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - this.unknownFields = builder.getUnknownFields(); - } - private MultiGetRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - - private static final MultiGetRequest defaultInstance; - public static MultiGetRequest getDefaultInstance() { - return defaultInstance; - } - - public MultiGetRequest getDefaultInstanceForType() { - return defaultInstance; - } - - private final com.google.protobuf.UnknownFieldSet unknownFields; - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private MultiGetRequest( - 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: { - 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)) { - get_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - get_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.PARSER, extensionRegistry)); - break; - } - case 24: { - bitField0_ |= 0x00000002; - closestRowBefore_ = input.readBool(); - break; - } - case 32: { - bitField0_ |= 0x00000004; - existenceOnly_ = input.readBool(); - 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_ & 0x00000002) == 0x00000002)) { - get_ = java.util.Collections.unmodifiableList(get_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetRequest_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.Builder.class); - } - - public static com.google.protobuf.Parser PARSER = - new com.google.protobuf.AbstractParser() { - public MultiGetRequest parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new MultiGetRequest(input, extensionRegistry); - } - }; - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - 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 .Get get = 2; - public static final int GET_FIELD_NUMBER = 2; - private java.util.List get_; - /** - * repeated .Get get = 2; - */ - public java.util.List getGetList() { - return get_; - } - /** - * repeated .Get get = 2; - */ - public java.util.List - getGetOrBuilderList() { - return get_; - } - /** - * repeated .Get get = 2; - */ - public int getGetCount() { - return get_.size(); - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get getGet(int index) { - return get_.get(index); - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder( - int index) { - return get_.get(index); - } - - // optional bool closest_row_before = 3; - public static final int CLOSEST_ROW_BEFORE_FIELD_NUMBER = 3; - private boolean closestRowBefore_; - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - public boolean hasClosestRowBefore() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - /** - * optional bool closest_row_before = 3; - * - *
-     * If the row to get doesn't exist, return the
-     * closest row before.
-     * 
- */ - public boolean getClosestRowBefore() { - return closestRowBefore_; - } - - // optional bool existence_only = 4; - public static final int EXISTENCE_ONLY_FIELD_NUMBER = 4; - private boolean existenceOnly_; - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - public boolean hasExistenceOnly() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - /** - * optional bool existence_only = 4; - * - *
-     * The result isn't asked for, just check for
-     * the existence. If closest_row_before specified,
-     * this will be ignored
-     * 
- */ - public boolean getExistenceOnly() { - return existenceOnly_; - } - - private void initFields() { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - get_ = java.util.Collections.emptyList(); - closestRowBefore_ = false; - existenceOnly_ = false; - } - 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 < getGetCount(); i++) { - if (!getGet(i).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.writeMessage(1, region_); - } - for (int i = 0; i < get_.size(); i++) { - output.writeMessage(2, get_.get(i)); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBool(3, closestRowBefore_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBool(4, existenceOnly_); - } - 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 - .computeMessageSize(1, region_); - } - for (int i = 0; i < get_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, get_.get(i)); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(3, closestRowBefore_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(4, existenceOnly_); - } - 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.MultiGetRequest)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest) obj; - - boolean result = true; - result = result && (hasRegion() == other.hasRegion()); - if (hasRegion()) { - result = result && getRegion() - .equals(other.getRegion()); - } - result = result && getGetList() - .equals(other.getGetList()); - result = result && (hasClosestRowBefore() == other.hasClosestRowBefore()); - if (hasClosestRowBefore()) { - result = result && (getClosestRowBefore() - == other.getClosestRowBefore()); - } - result = result && (hasExistenceOnly() == other.hasExistenceOnly()); - if (hasExistenceOnly()) { - result = result && (getExistenceOnly() - == other.getExistenceOnly()); - } - 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 (hasRegion()) { - hash = (37 * hash) + REGION_FIELD_NUMBER; - hash = (53 * hash) + getRegion().hashCode(); - } - if (getGetCount() > 0) { - hash = (37 * hash) + GET_FIELD_NUMBER; - hash = (53 * hash) + getGetList().hashCode(); - } - if (hasClosestRowBefore()) { - hash = (37 * hash) + CLOSEST_ROW_BEFORE_FIELD_NUMBER; - hash = (53 * hash) + hashBoolean(getClosestRowBefore()); - } - if (hasExistenceOnly()) { - hash = (37 * hash) + EXISTENCE_ONLY_FIELD_NUMBER; - hash = (53 * hash) + hashBoolean(getExistenceOnly()); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest 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.MultiGetRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest 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.MultiGetRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return PARSER.parseFrom(input); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest 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.MultiGetRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return PARSER.parseDelimitedFrom(input); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest 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.MultiGetRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return PARSER.parseFrom(input); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest 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.MultiGetRequest 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 MultiGetRequest} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetRequest_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetRequest_fieldAccessorTable - .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.Builder.class); - } - - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getRegionFieldBuilder(); - getGetFieldBuilder(); - } - } - private static Builder create() { - return new Builder(); - } - - 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 (getBuilder_ == null) { - get_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - } else { - getBuilder_.clear(); - } - closestRowBefore_ = false; - bitField0_ = (bitField0_ & ~0x00000004); - existenceOnly_ = false; - bitField0_ = (bitField0_ & ~0x00000008); - 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_MultiGetRequest_descriptor; - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.getDefaultInstance(); - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest(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 (getBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002)) { - get_ = java.util.Collections.unmodifiableList(get_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.get_ = get_; - } else { - result.get_ = getBuilder_.build(); - } - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000002; - } - result.closestRowBefore_ = closestRowBefore_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000004; - } - result.existenceOnly_ = existenceOnly_; - 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.MultiGetRequest) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.getDefaultInstance()) return this; - if (other.hasRegion()) { - mergeRegion(other.getRegion()); - } - if (getBuilder_ == null) { - if (!other.get_.isEmpty()) { - if (get_.isEmpty()) { - get_ = other.get_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureGetIsMutable(); - get_.addAll(other.get_); - } - onChanged(); - } - } else { - if (!other.get_.isEmpty()) { - if (getBuilder_.isEmpty()) { - getBuilder_.dispose(); - getBuilder_ = null; - get_ = other.get_; - bitField0_ = (bitField0_ & ~0x00000002); - getBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getGetFieldBuilder() : null; - } else { - getBuilder_.addAllMessages(other.get_); - } - } - } - if (other.hasClosestRowBefore()) { - setClosestRowBefore(other.getClosestRowBefore()); - } - if (other.hasExistenceOnly()) { - setExistenceOnly(other.getExistenceOnly()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - if (!hasRegion()) { - - return false; - } - if (!getRegion().isInitialized()) { - - return false; - } - for (int i = 0; i < getGetCount(); i++) { - if (!getGet(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.MultiGetRequest parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest) 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 regionBuilder_; - } - - // repeated .Get get = 2; - private java.util.List get_ = - java.util.Collections.emptyList(); - private void ensureGetIsMutable() { - if (!((bitField0_ & 0x00000002) == 0x00000002)) { - get_ = new java.util.ArrayList(get_); - bitField0_ |= 0x00000002; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - 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_; - - /** - * repeated .Get get = 2; - */ - public java.util.List getGetList() { - if (getBuilder_ == null) { - return java.util.Collections.unmodifiableList(get_); - } else { - return getBuilder_.getMessageList(); - } - } - /** - * repeated .Get get = 2; - */ - public int getGetCount() { - if (getBuilder_ == null) { - return get_.size(); - } else { - return getBuilder_.getCount(); - } - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get getGet(int index) { - if (getBuilder_ == null) { - return get_.get(index); - } else { - return getBuilder_.getMessage(index); - } - } - /** - * repeated .Get get = 2; - */ - public Builder setGet( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get value) { - if (getBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureGetIsMutable(); - get_.set(index, value); - onChanged(); - } else { - getBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder setGet( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder builderForValue) { - if (getBuilder_ == null) { - ensureGetIsMutable(); - get_.set(index, builderForValue.build()); - onChanged(); - } else { - getBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder addGet(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get value) { - if (getBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureGetIsMutable(); - get_.add(value); - onChanged(); - } else { - getBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder addGet( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get value) { - if (getBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureGetIsMutable(); - get_.add(index, value); - onChanged(); - } else { - getBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder addGet( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder builderForValue) { - if (getBuilder_ == null) { - ensureGetIsMutable(); - get_.add(builderForValue.build()); - onChanged(); - } else { - getBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder addGet( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder builderForValue) { - if (getBuilder_ == null) { - ensureGetIsMutable(); - get_.add(index, builderForValue.build()); - onChanged(); - } else { - getBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder addAllGet( - java.lang.Iterable values) { - if (getBuilder_ == null) { - ensureGetIsMutable(); - super.addAll(values, get_); - onChanged(); - } else { - getBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder clearGet() { - if (getBuilder_ == null) { - get_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - } else { - getBuilder_.clear(); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public Builder removeGet(int index) { - if (getBuilder_ == null) { - ensureGetIsMutable(); - get_.remove(index); - onChanged(); - } else { - getBuilder_.remove(index); - } - return this; - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder getGetBuilder( - int index) { - return getGetFieldBuilder().getBuilder(index); - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder( - int index) { - if (getBuilder_ == null) { - return get_.get(index); } else { - return getBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .Get get = 2; - */ - public java.util.List - getGetOrBuilderList() { - if (getBuilder_ != null) { - return getBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(get_); - } - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder addGetBuilder() { - return getGetFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance()); - } - /** - * repeated .Get get = 2; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.Builder addGetBuilder( - int index) { - return getGetFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance()); - } - /** - * repeated .Get get = 2; - */ - public java.util.List - getGetBuilderList() { - return getGetFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - 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> - getGetFieldBuilder() { - if (getBuilder_ == null) { - getBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - 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>( - get_, - ((bitField0_ & 0x00000002) == 0x00000002), - getParentForChildren(), - isClean()); - get_ = null; - } - return getBuilder_; - } - - // optional bool closest_row_before = 3; - private boolean closestRowBefore_ ; - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public boolean hasClosestRowBefore() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public boolean getClosestRowBefore() { - return closestRowBefore_; - } - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public Builder setClosestRowBefore(boolean value) { - bitField0_ |= 0x00000004; - closestRowBefore_ = value; - onChanged(); - return this; - } - /** - * optional bool closest_row_before = 3; - * - *
-       * If the row to get doesn't exist, return the
-       * closest row before.
-       * 
- */ - public Builder clearClosestRowBefore() { - bitField0_ = (bitField0_ & ~0x00000004); - closestRowBefore_ = false; - onChanged(); - return this; - } - - // optional bool existence_only = 4; - private boolean existenceOnly_ ; - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public boolean hasExistenceOnly() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public boolean getExistenceOnly() { - return existenceOnly_; - } - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public Builder setExistenceOnly(boolean value) { - bitField0_ |= 0x00000008; - existenceOnly_ = value; - onChanged(); - return this; - } - /** - * optional bool existence_only = 4; - * - *
-       * The result isn't asked for, just check for
-       * the existence. If closest_row_before specified,
-       * this will be ignored
-       * 
- */ - public Builder clearExistenceOnly() { - bitField0_ = (bitField0_ & ~0x00000008); - existenceOnly_ = false; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:MultiGetRequest) - } - - static { - defaultInstance = new MultiGetRequest(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:MultiGetRequest) - } - public interface GetResponseOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -5913,24 +4822,6 @@ * optional .Result result = 1; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder(); - - // optional bool exists = 2; - /** - * optional bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - boolean hasExists(); - /** - * optional bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - boolean getExists(); } /** * Protobuf type {@code GetResponse} @@ -5996,11 +4887,6 @@ bitField0_ |= 0x00000001; break; } - case 16: { - bitField0_ |= 0x00000002; - exists_ = input.readBool(); - break; - } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -6063,33 +4949,8 @@ return result_; } - // optional bool exists = 2; - public static final int EXISTS_FIELD_NUMBER = 2; - private boolean exists_; - /** - * optional bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - public boolean hasExists() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - /** - * optional bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - public boolean getExists() { - return exists_; - } - private void initFields() { result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); - exists_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -6106,9 +4967,6 @@ if (((bitField0_ & 0x00000001) == 0x00000001)) { output.writeMessage(1, result_); } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBool(2, exists_); - } getUnknownFields().writeTo(output); } @@ -6122,10 +4980,6 @@ size += com.google.protobuf.CodedOutputStream .computeMessageSize(1, result_); } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(2, exists_); - } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -6154,11 +5008,6 @@ result = result && getResult() .equals(other.getResult()); } - result = result && (hasExists() == other.hasExists()); - if (hasExists()) { - result = result && (getExists() - == other.getExists()); - } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -6176,10 +5025,6 @@ hash = (37 * hash) + RESULT_FIELD_NUMBER; hash = (53 * hash) + getResult().hashCode(); } - if (hasExists()) { - hash = (37 * hash) + EXISTS_FIELD_NUMBER; - hash = (53 * hash) + hashBoolean(getExists()); - } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -6296,8 +5141,6 @@ resultBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - exists_ = false; - bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -6334,10 +5177,6 @@ } else { result.result_ = resultBuilder_.build(); } - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.exists_ = exists_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -6357,9 +5196,6 @@ if (other.hasResult()) { mergeResult(other.getResult()); } - if (other.hasExists()) { - setExists(other.getExists()); - } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -6504,55 +5340,6 @@ return resultBuilder_; } - // optional bool exists = 2; - private boolean exists_ ; - /** - * optional bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public boolean hasExists() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - /** - * optional bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public boolean getExists() { - return exists_; - } - /** - * optional bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public Builder setExists(boolean value) { - bitField0_ |= 0x00000002; - exists_ = value; - onChanged(); - return this; - } - /** - * optional bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public Builder clearExists() { - bitField0_ = (bitField0_ & ~0x00000002); - exists_ = false; - onChanged(); - return this; - } - // @@protoc_insertion_point(builder_scope:GetResponse) } @@ -6564,927 +5351,6 @@ // @@protoc_insertion_point(class_scope:GetResponse) } - public interface MultiGetResponseOrBuilder - extends com.google.protobuf.MessageOrBuilder { - - // repeated .Result result = 1; - /** - * repeated .Result result = 1; - */ - java.util.List - getResultList(); - /** - * repeated .Result result = 1; - */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResult(int index); - /** - * repeated .Result result = 1; - */ - int getResultCount(); - /** - * repeated .Result result = 1; - */ - java.util.List - getResultOrBuilderList(); - /** - * repeated .Result result = 1; - */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder( - int index); - - // repeated bool exists = 2; - /** - * repeated bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - java.util.List getExistsList(); - /** - * repeated bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - int getExistsCount(); - /** - * repeated bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - boolean getExists(int index); - } - /** - * Protobuf type {@code MultiGetResponse} - */ - public static final class MultiGetResponse extends - com.google.protobuf.GeneratedMessage - implements MultiGetResponseOrBuilder { - // Use MultiGetResponse.newBuilder() to construct. - private MultiGetResponse(com.google.protobuf.GeneratedMessage.Builder builder) { - super(builder); - this.unknownFields = builder.getUnknownFields(); - } - private MultiGetResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - - private static final MultiGetResponse defaultInstance; - public static MultiGetResponse getDefaultInstance() { - return defaultInstance; - } - - public MultiGetResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private final com.google.protobuf.UnknownFieldSet unknownFields; - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; - } - private MultiGetResponse( - 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)) { - result_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000001; - } - result_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.PARSER, extensionRegistry)); - break; - } - case 16: { - if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) { - exists_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - exists_.add(input.readBool()); - break; - } - case 18: { - int length = input.readRawVarint32(); - int limit = input.pushLimit(length); - if (!((mutable_bitField0_ & 0x00000002) == 0x00000002) && input.getBytesUntilLimit() > 0) { - exists_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - while (input.getBytesUntilLimit() > 0) { - exists_.add(input.readBool()); - } - input.popLimit(limit); - 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)) { - result_ = java.util.Collections.unmodifiableList(result_); - } - if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { - exists_ = java.util.Collections.unmodifiableList(exists_); - } - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetResponse_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.Builder.class); - } - - public static com.google.protobuf.Parser PARSER = - new com.google.protobuf.AbstractParser() { - public MultiGetResponse parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new MultiGetResponse(input, extensionRegistry); - } - }; - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - // repeated .Result result = 1; - public static final int RESULT_FIELD_NUMBER = 1; - private java.util.List result_; - /** - * repeated .Result result = 1; - */ - public java.util.List getResultList() { - return result_; - } - /** - * repeated .Result result = 1; - */ - public java.util.List - getResultOrBuilderList() { - return result_; - } - /** - * repeated .Result result = 1; - */ - public int getResultCount() { - return result_.size(); - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResult(int index) { - return result_.get(index); - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder( - int index) { - return result_.get(index); - } - - // repeated bool exists = 2; - public static final int EXISTS_FIELD_NUMBER = 2; - private java.util.List exists_; - /** - * repeated bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - public java.util.List - getExistsList() { - return exists_; - } - /** - * repeated bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - public int getExistsCount() { - return exists_.size(); - } - /** - * repeated bool exists = 2; - * - *
-     * used for Get to check existence only
-     * 
- */ - public boolean getExists(int index) { - return exists_.get(index); - } - - private void initFields() { - result_ = java.util.Collections.emptyList(); - exists_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - memoizedIsInitialized = 1; - return true; - } - - 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 < exists_.size(); i++) { - output.writeBool(2, exists_.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 < result_.size(); i++) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, result_.get(i)); - } - { - int dataSize = 0; - dataSize = 1 * getExistsList().size(); - size += dataSize; - size += 1 * getExistsList().size(); - } - 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.MultiGetResponse)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse) obj; - - boolean result = true; - result = result && getResultList() - .equals(other.getResultList()); - result = result && getExistsList() - .equals(other.getExistsList()); - 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 (getResultCount() > 0) { - hash = (37 * hash) + RESULT_FIELD_NUMBER; - hash = (53 * hash) + getResultList().hashCode(); - } - if (getExistsCount() > 0) { - hash = (37 * hash) + EXISTS_FIELD_NUMBER; - hash = (53 * hash) + getExistsList().hashCode(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - memoizedHashCode = hash; - return hash; - } - - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse 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.MultiGetResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse 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.MultiGetResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return PARSER.parseFrom(input); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse 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.MultiGetResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return PARSER.parseDelimitedFrom(input); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse 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.MultiGetResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return PARSER.parseFrom(input); - } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse 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.MultiGetResponse 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 MultiGetResponse} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetResponse_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiGetResponse_fieldAccessorTable - .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.Builder.class); - } - - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.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(); - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - if (resultBuilder_ == null) { - result_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - } else { - resultBuilder_.clear(); - } - exists_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - 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_MultiGetResponse_descriptor; - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance(); - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse(this); - int from_bitField0_ = bitField0_; - if (resultBuilder_ == null) { - if (((bitField0_ & 0x00000001) == 0x00000001)) { - result_ = java.util.Collections.unmodifiableList(result_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.result_ = result_; - } else { - result.result_ = resultBuilder_.build(); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - exists_ = java.util.Collections.unmodifiableList(exists_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.exists_ = exists_; - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse other) { - if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance()) return this; - if (resultBuilder_ == null) { - if (!other.result_.isEmpty()) { - if (result_.isEmpty()) { - result_ = other.result_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensureResultIsMutable(); - result_.addAll(other.result_); - } - onChanged(); - } - } else { - if (!other.result_.isEmpty()) { - if (resultBuilder_.isEmpty()) { - resultBuilder_.dispose(); - resultBuilder_ = null; - result_ = other.result_; - bitField0_ = (bitField0_ & ~0x00000001); - resultBuilder_ = - com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getResultFieldBuilder() : null; - } else { - resultBuilder_.addAllMessages(other.result_); - } - } - } - if (!other.exists_.isEmpty()) { - if (exists_.isEmpty()) { - exists_ = other.exists_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureExistsIsMutable(); - exists_.addAll(other.exists_); - } - onChanged(); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - 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.MultiGetResponse parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse) e.getUnfinishedMessage(); - throw e; - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; - } - private int bitField0_; - - // repeated .Result result = 1; - private java.util.List result_ = - java.util.Collections.emptyList(); - private void ensureResultIsMutable() { - if (!((bitField0_ & 0x00000001) == 0x00000001)) { - result_ = new java.util.ArrayList(result_); - bitField0_ |= 0x00000001; - } - } - - private com.google.protobuf.RepeatedFieldBuilder< - 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_; - - /** - * repeated .Result result = 1; - */ - public java.util.List getResultList() { - if (resultBuilder_ == null) { - return java.util.Collections.unmodifiableList(result_); - } else { - return resultBuilder_.getMessageList(); - } - } - /** - * repeated .Result result = 1; - */ - public int getResultCount() { - if (resultBuilder_ == null) { - return result_.size(); - } else { - return resultBuilder_.getCount(); - } - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getResult(int index) { - if (resultBuilder_ == null) { - return result_.get(index); - } else { - return resultBuilder_.getMessage(index); - } - } - /** - * repeated .Result result = 1; - */ - public Builder setResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (resultBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureResultIsMutable(); - result_.set(index, value); - onChanged(); - } else { - resultBuilder_.setMessage(index, value); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder setResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder builderForValue) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.set(index, builderForValue.build()); - onChanged(); - } else { - resultBuilder_.setMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder addResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (resultBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureResultIsMutable(); - result_.add(value); - onChanged(); - } else { - resultBuilder_.addMessage(value); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder addResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (resultBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - ensureResultIsMutable(); - result_.add(index, value); - onChanged(); - } else { - resultBuilder_.addMessage(index, value); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder addResult( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder builderForValue) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.add(builderForValue.build()); - onChanged(); - } else { - resultBuilder_.addMessage(builderForValue.build()); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder addResult( - int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder builderForValue) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.add(index, builderForValue.build()); - onChanged(); - } else { - resultBuilder_.addMessage(index, builderForValue.build()); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder addAllResult( - java.lang.Iterable values) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - super.addAll(values, result_); - onChanged(); - } else { - resultBuilder_.addAllMessages(values); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder clearResult() { - if (resultBuilder_ == null) { - result_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - onChanged(); - } else { - resultBuilder_.clear(); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public Builder removeResult(int index) { - if (resultBuilder_ == null) { - ensureResultIsMutable(); - result_.remove(index); - onChanged(); - } else { - resultBuilder_.remove(index); - } - return this; - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder getResultBuilder( - int index) { - return getResultFieldBuilder().getBuilder(index); - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder( - int index) { - if (resultBuilder_ == null) { - return result_.get(index); } else { - return resultBuilder_.getMessageOrBuilder(index); - } - } - /** - * repeated .Result result = 1; - */ - public java.util.List - getResultOrBuilderList() { - if (resultBuilder_ != null) { - return resultBuilder_.getMessageOrBuilderList(); - } else { - return java.util.Collections.unmodifiableList(result_); - } - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder addResultBuilder() { - return getResultFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance()); - } - /** - * repeated .Result result = 1; - */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder addResultBuilder( - int index) { - return getResultFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance()); - } - /** - * repeated .Result result = 1; - */ - public java.util.List - getResultBuilderList() { - return getResultFieldBuilder().getBuilderList(); - } - private com.google.protobuf.RepeatedFieldBuilder< - 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.RepeatedFieldBuilder< - 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_, - ((bitField0_ & 0x00000001) == 0x00000001), - getParentForChildren(), - isClean()); - result_ = null; - } - return resultBuilder_; - } - - // repeated bool exists = 2; - private java.util.List exists_ = java.util.Collections.emptyList(); - private void ensureExistsIsMutable() { - if (!((bitField0_ & 0x00000002) == 0x00000002)) { - exists_ = new java.util.ArrayList(exists_); - bitField0_ |= 0x00000002; - } - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public java.util.List - getExistsList() { - return java.util.Collections.unmodifiableList(exists_); - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public int getExistsCount() { - return exists_.size(); - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public boolean getExists(int index) { - return exists_.get(index); - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public Builder setExists( - int index, boolean value) { - ensureExistsIsMutable(); - exists_.set(index, value); - onChanged(); - return this; - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public Builder addExists(boolean value) { - ensureExistsIsMutable(); - exists_.add(value); - onChanged(); - return this; - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public Builder addAllExists( - java.lang.Iterable values) { - ensureExistsIsMutable(); - super.addAll(values, exists_); - onChanged(); - return this; - } - /** - * repeated bool exists = 2; - * - *
-       * used for Get to check existence only
-       * 
- */ - public Builder clearExists() { - exists_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:MultiGetResponse) - } - - static { - defaultInstance = new MultiGetResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:MultiGetResponse) - } - public interface ConditionOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -23574,62 +21440,80 @@ // @@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; } @@ -23639,7 +21523,7 @@ getUnknownFields() { return this.unknownFields; } - private MultiAction( + private Action( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -23662,9 +21546,14 @@ } 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); @@ -23672,12 +21561,12 @@ 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); @@ -23685,7 +21574,7 @@ subBuilder.mergeFrom(get_); get_ = subBuilder.buildPartial(); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; break; } } @@ -23702,77 +21591,104 @@ } 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(); } @@ -23801,11 +21717,14 @@ 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); } @@ -23817,12 +21736,16 @@ 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; return size; @@ -23840,12 +21763,17 @@ 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() @@ -23869,6 +21797,10 @@ } 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(); @@ -23882,53 +21814,53 @@ 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 { @@ -23937,7 +21869,7 @@ 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); } @@ -23949,30 +21881,28 @@ 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(); } @@ -23994,18 +21924,20 @@ 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; } @@ -24015,35 +21947,39 @@ 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_; @@ -24056,16 +21992,19 @@ } 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()); } @@ -24096,11 +22035,11 @@ 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) { @@ -24111,18 +22050,71 @@ } 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) { @@ -24132,7 +22124,7 @@ } } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public Builder setMutation(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) { if (mutationBuilder_ == null) { @@ -24144,11 +22136,11 @@ } 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) { @@ -24158,15 +22150,15 @@ } 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(); @@ -24177,11 +22169,11 @@ } else { mutationBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .MutationProto mutation = 1; + * optional .MutationProto mutation = 2; */ public Builder clearMutation() { if (mutationBuilder_ == null) { @@ -24190,19 +22182,19 @@ } 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) { @@ -24212,7 +22204,7 @@ } } /** - * 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> @@ -24228,18 +22220,18 @@ 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) { @@ -24249,7 +22241,7 @@ } } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public Builder setGet(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get value) { if (getBuilder_ == null) { @@ -24261,11 +22253,11 @@ } 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) { @@ -24275,15 +22267,15 @@ } 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(); @@ -24294,11 +22286,11 @@ } else { getBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .Get get = 2; + * optional .Get get = 3; */ public Builder clearGet() { if (getBuilder_ == null) { @@ -24307,19 +22299,19 @@ } 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) { @@ -24329,7 +22321,7 @@ } } /** - * 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> @@ -24345,77 +22337,1167 @@ 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(); /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getValue(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); /** - * optional .Result value = 1; + * required .RegionSpecifier region = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getValueOrBuilder(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); - // optional .NameBytesPair exception = 2; + // optional bool atomic = 2; /** - * optional .NameBytesPair exception = 2; + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
*/ + boolean hasAtomic(); + /** + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
+ */ + boolean getAtomic(); + + // repeated .Action action = 3; + /** + * repeated .Action action = 3; + */ + java.util.List + getActionList(); + /** + * repeated .Action action = 3; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action getAction(int index); + /** + * repeated .Action action = 3; + */ + 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 RegionAction} + * + *
+   **
+   * Actions to run against a Region.
+   * 
+ */ + public static final class RegionAction extends + com.google.protobuf.GeneratedMessage + implements RegionActionOrBuilder { + // Use RegionAction.newBuilder() to construct. + private RegionAction(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private RegionAction(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final RegionAction defaultInstance; + public static RegionAction getDefaultInstance() { + return defaultInstance; + } + + public RegionAction getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RegionAction( + 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: { + 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 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; + } + } + } + } 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_ & 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_RegionAction_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionAction_fieldAccessorTable + .ensureFieldAccessorsInitialized( + 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 RegionAction parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RegionAction(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + 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_; + } + + // optional bool atomic = 2; + public static final int ATOMIC_FIELD_NUMBER = 2; + private boolean atomic_; + /** + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
+ */ + public boolean hasAtomic() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bool atomic = 2; + * + *
+     * When set, run mutations as atomic unit.
+     * 
+ */ + 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_; + } + /** + * repeated .Action action = 3; + */ + 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() { + 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 (!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; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, region_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(2, atomic_); + } + for (int i = 0; i < action_.size(); i++) { + output.writeMessage(3, action_.get(i)); + } + 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 + .computeMessageSize(1, region_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .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; + 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.RegionAction)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction) obj; + + boolean result = true; + result = result && (hasRegion() == other.hasRegion()); + if (hasRegion()) { + result = result && getRegion() + .equals(other.getRegion()); + } + 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; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegion()) { + hash = (37 * hash) + REGION_FIELD_NUMBER; + hash = (53 * hash) + getRegion().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.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.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.RegionAction parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + 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.RegionAction parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + 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.RegionAction parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + 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.RegionAction parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + 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 { + 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.RegionAction 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 RegionAction} + * + *
+     **
+     * 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.RegionActionOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + 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_RegionAction_fieldAccessorTable + .ensureFieldAccessorsInitialized( + 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.RegionAction.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRegionFieldBuilder(); + getActionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + } else { + regionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + atomic_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + if (actionBuilder_ == null) { + action_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + } else { + actionBuilder_.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_RegionAction_descriptor; + } + + 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.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.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 (regionBuilder_ == null) { + result.region_ = region_; + } else { + result.region_ = regionBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.atomic_ = atomic_; + if (actionBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { + action_ = java.util.Collections.unmodifiableList(action_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.action_ = action_; + } else { + result.action_ = actionBuilder_.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.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.RegionAction other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.getDefaultInstance()) return this; + if (other.hasRegion()) { + mergeRegion(other.getRegion()); + } + 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 (!hasRegion()) { + + return false; + } + if (!getRegion().isInitialized()) { + + return false; + } + for (int i = 0; i < getActionCount(); i++) { + if (!getAction(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.RegionAction parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction) 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 regionBuilder_; + } + + // optional bool atomic = 2; + private boolean atomic_ ; + /** + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
+ */ + public boolean hasAtomic() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
+ */ + public boolean getAtomic() { + return atomic_; + } + /** + * optional bool atomic = 2; + * + *
+       * When set, run mutations as atomic unit.
+       * 
+ */ + public Builder setAtomic(boolean value) { + bitField0_ |= 0x00000002; + atomic_ = value; + onChanged(); + return this; + } + /** + * 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 = 2; + * optional .NameBytesPair exception = 3; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException(); /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder(); } /** - * Protobuf type {@code ActionResult} + * Protobuf type {@code ResultOrException} * *
    **
-   * 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.
    * 
*/ - public static final class ActionResult extends + public static final class ResultOrException extends com.google.protobuf.GeneratedMessage - implements ActionResultOrBuilder { - // Use ActionResult.newBuilder() to construct. - private ActionResult(com.google.protobuf.GeneratedMessage.Builder builder) { + implements ResultOrExceptionOrBuilder { + // Use ResultOrException.newBuilder() to construct. + private ResultOrException(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); this.unknownFields = builder.getUnknownFields(); } - private ActionResult(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private ResultOrException(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - private static final ActionResult defaultInstance; - public static ActionResult getDefaultInstance() { + private static final ResultOrException defaultInstance; + public static ResultOrException getDefaultInstance() { return defaultInstance; } - public ActionResult getDefaultInstanceForType() { + public ResultOrException getDefaultInstanceForType() { return defaultInstance; } @@ -24425,7 +23507,7 @@ getUnknownFields() { return this.unknownFields; } - private ActionResult( + private ResultOrException( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -24448,22 +23530,27 @@ } break; } - case 10: { + case 8: { + bitField0_ |= 0x00000001; + index_ = input.readUInt32(); + break; + } + case 18: { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder subBuilder = null; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - subBuilder = value_.toBuilder(); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = result_.toBuilder(); } - value_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.PARSER, extensionRegistry); + result_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.PARSER, extensionRegistry); if (subBuilder != null) { - subBuilder.mergeFrom(value_); - value_ = subBuilder.buildPartial(); + subBuilder.mergeFrom(result_); + result_ = subBuilder.buildPartial(); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; break; } - case 18: { + case 26: { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = null; - if (((bitField0_ & 0x00000002) == 0x00000002)) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { subBuilder = exception_.toBuilder(); } exception_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.PARSER, extensionRegistry); @@ -24471,7 +23558,7 @@ subBuilder.mergeFrom(exception_); exception_ = subBuilder.buildPartial(); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; break; } } @@ -24488,78 +23575,105 @@ } 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_ResultOrException_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_ResultOrException_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.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 ActionResult parsePartialFrom( + 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 ActionResult(input, extensionRegistry); + return new ResultOrException(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_; + // optional uint32 index = 1; + public static final int INDEX_FIELD_NUMBER = 1; + private int index_; /** - * optional .Result value = 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.
+     * 
*/ - public boolean hasValue() { + public boolean hasIndex() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * optional .Result value = 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.
+     * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getValue() { - return value_; + 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 value = 1; + * optional .Result result = 2; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getValueOrBuilder() { - return value_; + 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 = 2; - public static final int EXCEPTION_FIELD_NUMBER = 2; + // 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 = 2; + * optional .NameBytesPair exception = 3; */ public boolean hasException() { - return ((bitField0_ & 0x00000002) == 0x00000002); + return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { return exception_; } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { return exception_; } private void initFields() { - value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + 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; @@ -24581,11 +23695,14 @@ throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, value_); + output.writeUInt32(1, index_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, exception_); + output.writeMessage(2, result_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, exception_); + } getUnknownFields().writeTo(output); } @@ -24597,12 +23714,16 @@ size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, value_); + .computeUInt32Size(1, index_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, exception_); + .computeMessageSize(2, result_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, exception_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -24620,17 +23741,22 @@ 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.ResultOrException)) { 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.ResultOrException other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException) obj; boolean result = true; - result = result && (hasValue() == other.hasValue()); - if (hasValue()) { - result = result && getValue() - .equals(other.getValue()); + 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() @@ -24649,10 +23775,14 @@ } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasValue()) { - hash = (37 * hash) + VALUE_FIELD_NUMBER; - hash = (53 * hash) + getValue().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(); @@ -24662,53 +23792,53 @@ return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult parseFrom( + 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.ActionResult parseFrom( + 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.ActionResult parseFrom(byte[] data) + 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.ActionResult parseFrom( + 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.ActionResult parseFrom(java.io.InputStream input) + 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.ActionResult parseFrom( + 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.ActionResult parseDelimitedFrom(java.io.InputStream input) + 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.ActionResult parseDelimitedFrom( + 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.ActionResult parseFrom( + 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.ActionResult parseFrom( + 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 { @@ -24717,7 +23847,7 @@ 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.ResultOrException prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -24729,34 +23859,31 @@ return builder; } /** - * Protobuf type {@code ActionResult} + * Protobuf type {@code ResultOrException} * *
      **
-     * 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.
      * 
*/ 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.ResultOrExceptionOrBuilder { 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_ResultOrException_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_ResultOrException_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.ResultOrException.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.Builder.class); } - // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -24768,7 +23895,7 @@ } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getValueFieldBuilder(); + getResultFieldBuilder(); getExceptionFieldBuilder(); } } @@ -24778,18 +23905,20 @@ public Builder clear() { super.clear(); - if (valueBuilder_ == null) { - value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + index_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + if (resultBuilder_ == null) { + result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); } else { - valueBuilder_.clear(); + resultBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000001); + bitField0_ = (bitField0_ & ~0x00000002); if (exceptionBuilder_ == null) { exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); } else { exceptionBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -24799,36 +23928,40 @@ 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_ResultOrException_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.ResultOrException getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException.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.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.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.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; } - if (valueBuilder_ == null) { - result.value_ = value_; - } else { - result.value_ = valueBuilder_.build(); - } + 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 { @@ -24840,19 +23973,22 @@ } 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.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.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.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()); } @@ -24874,11 +24010,11 @@ 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.ResultOrException 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.ResultOrException) e.getUnfinishedMessage(); throw e; } finally { if (parsedMessage != null) { @@ -24889,135 +24025,188 @@ } 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(); - 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_; + // optional uint32 index = 1; + private int index_ ; /** - * optional .Result value = 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.
+       * 
*/ - public boolean hasValue() { + public boolean hasIndex() { return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * optional .Result value = 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.
+       * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result getValue() { - if (valueBuilder_ == null) { - return value_; + 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 valueBuilder_.getMessage(); + return resultBuilder_.getMessage(); } } /** - * optional .Result value = 1; + * optional .Result result = 2; */ - public Builder setValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (valueBuilder_ == null) { + public Builder setResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { + if (resultBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - value_ = value; + result_ = value; onChanged(); } else { - valueBuilder_.setMessage(value); + resultBuilder_.setMessage(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .Result value = 1; + * optional .Result result = 2; */ - public Builder setValue( + public Builder setResult( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder builderForValue) { - if (valueBuilder_ == null) { - value_ = builderForValue.build(); + if (resultBuilder_ == null) { + result_ = builderForValue.build(); onChanged(); } else { - valueBuilder_.setMessage(builderForValue.build()); + resultBuilder_.setMessage(builderForValue.build()); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .Result value = 1; + * optional .Result result = 2; */ - public Builder mergeValue(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result value) { - if (valueBuilder_ == 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(); + 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 { - value_ = value; + result_ = value; } onChanged(); } else { - valueBuilder_.mergeFrom(value); + resultBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * optional .Result value = 1; + * optional .Result result = 2; */ - public Builder clearValue() { - if (valueBuilder_ == null) { - value_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); + public Builder clearResult() { + if (resultBuilder_ == null) { + result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); onChanged(); } else { - valueBuilder_.clear(); + resultBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000001); + bitField0_ = (bitField0_ & ~0x00000002); return this; } /** - * optional .Result value = 1; + * optional .Result result = 2; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder getValueBuilder() { - bitField0_ |= 0x00000001; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.Builder getResultBuilder() { + bitField0_ |= 0x00000002; onChanged(); - return getValueFieldBuilder().getBuilder(); + return getResultFieldBuilder().getBuilder(); } /** - * optional .Result value = 1; + * optional .Result result = 2; */ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getValueOrBuilder() { - if (valueBuilder_ != null) { - return valueBuilder_.getMessageOrBuilder(); + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrBuilder getResultOrBuilder() { + if (resultBuilder_ != null) { + return resultBuilder_.getMessageOrBuilder(); } else { - return value_; + return result_; } } /** - * optional .Result value = 1; + * 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> - getValueFieldBuilder() { - if (valueBuilder_ == null) { - valueBuilder_ = new com.google.protobuf.SingleFieldBuilder< + 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>( - value_, + result_, getParentForChildren(), isClean()); - value_ = null; + result_ = null; } - return valueBuilder_; + return resultBuilder_; } - // optional .NameBytesPair exception = 2; + // 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 = 2; + * optional .NameBytesPair exception = 3; */ public boolean hasException() { - return ((bitField0_ & 0x00000002) == 0x00000002); + return ((bitField0_ & 0x00000004) == 0x00000004); } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { if (exceptionBuilder_ == null) { @@ -25027,7 +24216,7 @@ } } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public Builder setException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { if (exceptionBuilder_ == null) { @@ -25039,11 +24228,11 @@ } else { exceptionBuilder_.setMessage(value); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public Builder setException( org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) { @@ -25053,15 +24242,15 @@ } else { exceptionBuilder_.setMessage(builderForValue.build()); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public Builder mergeException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { if (exceptionBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002) && + 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(); @@ -25072,11 +24261,11 @@ } else { exceptionBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; return this; } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public Builder clearException() { if (exceptionBuilder_ == null) { @@ -25085,19 +24274,19 @@ } else { exceptionBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000002); + bitField0_ = (bitField0_ & ~0x00000004); return this; } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getExceptionBuilder() { - bitField0_ |= 0x00000002; + bitField0_ |= 0x00000004; onChanged(); return getExceptionFieldBuilder().getBuilder(); } /** - * optional .NameBytesPair exception = 2; + * optional .NameBytesPair exception = 3; */ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { if (exceptionBuilder_ != null) { @@ -25107,7 +24296,7 @@ } } /** - * optional .NameBytesPair exception = 2; + * 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> @@ -25123,97 +24312,95 @@ return exceptionBuilder_; } - // @@protoc_insertion_point(builder_scope:ActionResult) + // @@protoc_insertion_point(builder_scope:ResultOrException) } static { - defaultInstance = new ActionResult(true); + defaultInstance = new ResultOrException(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:ActionResult) + // @@protoc_insertion_point(class_scope:ResultOrException) } - public interface MultiRequestOrBuilder + public interface RegionActionResultOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required .RegionSpecifier region = 1; + // repeated .ResultOrException resultOrException = 1; /** - * required .RegionSpecifier region = 1; + * repeated .ResultOrException resultOrException = 1; */ - boolean hasRegion(); + java.util.List + getResultOrExceptionList(); /** - * required .RegionSpecifier region = 1; + * repeated .ResultOrException resultOrException = 1; */ - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException getResultOrException(int index); /** - * required .RegionSpecifier region = 1; + * repeated .ResultOrException resultOrException = 1; */ - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); - - // repeated .MultiAction action = 2; + int getResultOrExceptionCount(); /** - * repeated .MultiAction action = 2; + * repeated .ResultOrException resultOrException = 1; */ - java.util.List - getActionList(); + java.util.List + getResultOrExceptionOrBuilderList(); /** - * repeated .MultiAction action = 2; + * repeated .ResultOrException resultOrException = 1; */ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction getAction(int index); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder getResultOrExceptionOrBuilder( + int index); + + // optional .NameBytesPair exception = 2; /** - * repeated .MultiAction action = 2; + * optional .NameBytesPair exception = 2; + * + *
+     * If the operation failed globally for this region, this exception is set
+     * 
*/ - int getActionCount(); + boolean hasException(); /** - * repeated .MultiAction action = 2; + * optional .NameBytesPair exception = 2; + * + *
+     * If the operation failed globally for this region, this exception is set
+     * 
*/ - java.util.List - getActionOrBuilderList(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException(); /** - * repeated .MultiAction action = 2; + * optional .NameBytesPair exception = 2; + * + *
+     * If the operation failed globally for this region, this exception is set
+     * 
*/ - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder getActionOrBuilder( - int index); - - // optional bool atomic = 3; - /** - * optional bool atomic = 3; - */ - boolean hasAtomic(); - /** - * optional bool atomic = 3; - */ - boolean getAtomic(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder(); } /** - * Protobuf type {@code MultiRequest} + * Protobuf type {@code RegionActionResult} * *
    **
-   * 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.
    * 
*/ - public static final class MultiRequest extends + public static final class RegionActionResult extends com.google.protobuf.GeneratedMessage - implements MultiRequestOrBuilder { - // Use MultiRequest.newBuilder() to construct. - private MultiRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + implements RegionActionResultOrBuilder { + // Use RegionActionResult.newBuilder() to construct. + private RegionActionResult(com.google.protobuf.GeneratedMessage.Builder builder) { super(builder); this.unknownFields = builder.getUnknownFields(); } - private MultiRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + private RegionActionResult(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - private static final MultiRequest defaultInstance; - public static MultiRequest getDefaultInstance() { + private static final RegionActionResult defaultInstance; + public static RegionActionResult getDefaultInstance() { return defaultInstance; } - public MultiRequest getDefaultInstanceForType() { + public RegionActionResult getDefaultInstanceForType() { return defaultInstance; } @@ -25223,7 +24410,7 @@ getUnknownFields() { return this.unknownFields; } - private MultiRequest( + private RegionActionResult( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { @@ -25247,31 +24434,26 @@ break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null; + 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; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = null; if (((bitField0_ & 0x00000001) == 0x00000001)) { - subBuilder = region_.toBuilder(); + subBuilder = exception_.toBuilder(); } - region_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry); + exception_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.PARSER, extensionRegistry); if (subBuilder != null) { - subBuilder.mergeFrom(region_); - region_ = subBuilder.buildPartial(); + subBuilder.mergeFrom(exception_); + exception_ = subBuilder.buildPartial(); } bitField0_ |= 0x00000001; break; } - case 18: { - if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) { - action_ = new java.util.ArrayList(); - mutable_bitField0_ |= 0x00000002; - } - action_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.PARSER, extensionRegistry)); - break; - } - case 24: { - bitField0_ |= 0x00000002; - atomic_ = input.readBool(); - break; - } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -25280,8 +24462,8 @@ 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)) { + resultOrException_ = java.util.Collections.unmodifiableList(resultOrException_); } this.unknownFields = unknownFields.build(); makeExtensionsImmutable(); @@ -25289,126 +24471,119 @@ } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiRequest_descriptor; + 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_MultiRequest_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_fieldAccessorTable .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.Builder.class); + 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 MultiRequest parsePartialFrom( + 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 MultiRequest(input, extensionRegistry); + return new RegionActionResult(input, extensionRegistry); } }; @java.lang.Override - public com.google.protobuf.Parser getParserForType() { + public com.google.protobuf.Parser getParserForType() { 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_; + // repeated .ResultOrException resultOrException = 1; + public static final int RESULTOREXCEPTION_FIELD_NUMBER = 1; + private java.util.List resultOrException_; /** - * required .RegionSpecifier region = 1; + * repeated .ResultOrException resultOrException = 1; */ - public boolean hasRegion() { - return ((bitField0_ & 0x00000001) == 0x00000001); + public java.util.List getResultOrExceptionList() { + return resultOrException_; } /** - * required .RegionSpecifier region = 1; + * repeated .ResultOrException resultOrException = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { - return region_; + public java.util.List + getResultOrExceptionOrBuilderList() { + return resultOrException_; } /** - * required .RegionSpecifier region = 1; + * repeated .ResultOrException resultOrException = 1; */ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { - return region_; + public int getResultOrExceptionCount() { + return resultOrException_.size(); } - - // repeated .MultiAction action = 2; - public static final int ACTION_FIELD_NUMBER = 2; - private java.util.List action_; /** - * repeated .MultiAction action = 2; + * repeated .ResultOrException resultOrException = 1; */ - public java.util.List getActionList() { - return action_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException getResultOrException(int index) { + return resultOrException_.get(index); } /** - * repeated .MultiAction action = 2; + * repeated .ResultOrException resultOrException = 1; */ - public java.util.List - getActionOrBuilderList() { - return action_; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrExceptionOrBuilder getResultOrExceptionOrBuilder( + int index) { + return resultOrException_.get(index); } + + // optional .NameBytesPair exception = 2; + public static final int EXCEPTION_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair exception_; /** - * repeated .MultiAction action = 2; + * optional .NameBytesPair exception = 2; + * + *
+     * If the operation failed globally for this region, this exception is set
+     * 
*/ - public int getActionCount() { - return action_.size(); + public boolean hasException() { + return ((bitField0_ & 0x00000001) == 0x00000001); } /** - * repeated .MultiAction action = 2; + * optional .NameBytesPair exception = 2; + * + *
+     * If the operation failed globally for this region, this exception is set
+     * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction getAction(int index) { - return action_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { + return exception_; } /** - * repeated .MultiAction action = 2; + * optional .NameBytesPair exception = 2; + * + *
+     * If the operation failed globally for this region, this exception is set
+     * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiActionOrBuilder getActionOrBuilder( - int index) { - return action_.get(index); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { + return exception_; } - // 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_; - } - private void initFields() { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); - action_ = java.util.Collections.emptyList(); - atomic_ = false; + resultOrException_ = java.util.Collections.emptyList(); + 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 (!hasRegion()) { - memoizedIsInitialized = 0; - return false; + for (int i = 0; i < getResultOrExceptionCount(); i++) { + if (!getResultOrException(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } - if (!getRegion().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getActionCount(); i++) { - if (!getAction(i).isInitialized()) { + if (hasException()) { + if (!getException().isInitialized()) { memoizedIsInitialized = 0; return false; } @@ -25420,15 +24595,12 @@ 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)); + } if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, region_); + output.writeMessage(2, exception_); } - for (int i = 0; i < action_.size(); i++) { - output.writeMessage(2, action_.get(i)); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBool(3, atomic_); - } getUnknownFields().writeTo(output); } @@ -25438,18 +24610,14 @@ if (size != -1) return size; size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { + for (int i = 0; i < resultOrException_.size(); i++) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, region_); + .computeMessageSize(1, resultOrException_.get(i)); } - for (int i = 0; i < action_.size(); i++) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, action_.get(i)); + .computeMessageSize(2, exception_); } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(3, atomic_); - } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -25467,24 +24635,19 @@ if (obj == this) { return true; } - if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)) { + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult)) { return super.equals(obj); } - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest) obj; + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult) obj; boolean result = true; - result = result && (hasRegion() == other.hasRegion()); - if (hasRegion()) { - result = result && getRegion() - .equals(other.getRegion()); + result = result && getResultOrExceptionList() + .equals(other.getResultOrExceptionList()); + result = result && (hasException() == other.hasException()); + if (hasException()) { + result = result && getException() + .equals(other.getException()); } - result = result && getActionList() - .equals(other.getActionList()); - result = result && (hasAtomic() == other.hasAtomic()); - if (hasAtomic()) { - result = result && (getAtomic() - == other.getAtomic()); - } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -25498,70 +24661,66 @@ } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasRegion()) { - hash = (37 * hash) + REGION_FIELD_NUMBER; - hash = (53 * hash) + getRegion().hashCode(); + if (getResultOrExceptionCount() > 0) { + hash = (37 * hash) + RESULTOREXCEPTION_FIELD_NUMBER; + hash = (53 * hash) + getResultOrExceptionList().hashCode(); } - if (getActionCount() > 0) { - hash = (37 * hash) + ACTION_FIELD_NUMBER; - hash = (53 * hash) + getActionList().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()); - } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; } - public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest parseFrom( + 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.MultiRequest parseFrom( + 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.MultiRequest parseFrom(byte[] data) + 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.MultiRequest parseFrom( + 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.MultiRequest parseFrom(java.io.InputStream input) + 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.MultiRequest parseFrom( + 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.MultiRequest parseDelimitedFrom(java.io.InputStream input) + 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.MultiRequest parseDelimitedFrom( + 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.MultiRequest parseFrom( + 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.MultiRequest parseFrom( + 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 { @@ -25570,7 +24729,7 @@ public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest prototype) { + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -25582,33 +24741,29 @@ return builder; } /** - * Protobuf type {@code MultiRequest} + * Protobuf type {@code RegionActionResult} * *
      **
-     * 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.
      * 
*/ public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequestOrBuilder { + 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_MultiRequest_descriptor; + 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_MultiRequest_fieldAccessorTable + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_fieldAccessorTable .ensureFieldAccessorsInitialized( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.Builder.class); + 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.MultiRequest.newBuilder() + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -25620,8 +24775,8 @@ } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getRegionFieldBuilder(); - getActionFieldBuilder(); + getResultOrExceptionFieldBuilder(); + getExceptionFieldBuilder(); } } private static Builder create() { @@ -25630,20 +24785,18 @@ public Builder clear() { super.clear(); - if (regionBuilder_ == null) { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + if (resultOrExceptionBuilder_ == null) { + resultOrException_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); } else { - regionBuilder_.clear(); + resultOrExceptionBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000001); - if (actionBuilder_ == null) { - action_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); + if (exceptionBuilder_ == null) { + exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); } else { - actionBuilder_.clear(); + exceptionBuilder_.clear(); } - atomic_ = false; - bitField0_ = (bitField0_ & ~0x00000004); + bitField0_ = (bitField0_ & ~0x00000002); return this; } @@ -25653,109 +24806,100 @@ public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiRequest_descriptor; + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_RegionActionResult_descriptor; } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest getDefaultInstanceForType() { - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); + 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.MultiRequest build() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest result = buildPartial(); + 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.MultiRequest buildPartial() { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest(this); + 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_; int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + if (resultOrExceptionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + resultOrException_ = java.util.Collections.unmodifiableList(resultOrException_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.resultOrException_ = resultOrException_; + } else { + result.resultOrException_ = resultOrExceptionBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000001; } - if (regionBuilder_ == null) { - result.region_ = region_; + if (exceptionBuilder_ == null) { + result.exception_ = exception_; } else { - result.region_ = regionBuilder_.build(); + result.exception_ = exceptionBuilder_.build(); } - if (actionBuilder_ == null) { - if (((bitField0_ & 0x00000002) == 0x00000002)) { - action_ = java.util.Collections.unmodifiableList(action_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.action_ = action_; - } else { - result.action_ = actionBuilder_.build(); - } - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000002; - } - result.atomic_ = atomic_; 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.MultiRequest) { - return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)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.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); + 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 { - ensureActionIsMutable(); - action_.addAll(other.action_); + ensureResultOrExceptionIsMutable(); + resultOrException_.addAll(other.resultOrException_); } onChanged(); } } else { - if (!other.action_.isEmpty()) { - if (actionBuilder_.isEmpty()) { - actionBuilder_.dispose(); - actionBuilder_ = null; - action_ = other.action_; - bitField0_ = (bitField0_ & ~0x00000002); - actionBuilder_ = + if (!other.resultOrException_.isEmpty()) { + if (resultOrExceptionBuilder_.isEmpty()) { + resultOrExceptionBuilder_.dispose(); + resultOrExceptionBuilder_ = null; + resultOrException_ = other.resultOrException_; + bitField0_ = (bitField0_ & ~0x00000001); + resultOrExceptionBuilder_ = com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? - getActionFieldBuilder() : null; + getResultOrExceptionFieldBuilder() : null; } else { - actionBuilder_.addAllMessages(other.action_); + resultOrExceptionBuilder_.addAllMessages(other.resultOrException_); } } } - if (other.hasAtomic()) { - setAtomic(other.getAtomic()); + if (other.hasException()) { + mergeException(other.getException()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasRegion()) { - - return false; + for (int i = 0; i < getResultOrExceptionCount(); i++) { + if (!getResultOrException(i).isInitialized()) { + + return false; + } } - if (!getRegion().isInitialized()) { - - return false; - } - for (int i = 0; i < getActionCount(); i++) { - if (!getAction(i).isInitialized()) { + if (hasException()) { + if (!getException().isInitialized()) { return false; } @@ -25767,11 +24911,11 @@ com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest parsedMessage = null; + 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.MultiRequest) e.getUnfinishedMessage(); + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult) e.getUnfinishedMessage(); throw e; } finally { if (parsedMessage != null) { @@ -25782,396 +24926,1138 @@ } 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(); + // 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); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + 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 { + resultOrExceptionBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + 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 { + resultOrExceptionBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + 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; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + 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 { + resultOrExceptionBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .ResultOrException resultOrException = 1; + */ + 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()); + resultOrException_ = null; + } + return resultOrExceptionBuilder_; + } + + // 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.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_; + 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_; /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public boolean hasRegion() { - return ((bitField0_ & 0x00000001) == 0x00000001); + public boolean hasException() { + return ((bitField0_ & 0x00000002) == 0x00000002); } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { - if (regionBuilder_ == null) { - return region_; + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getException() { + if (exceptionBuilder_ == null) { + return exception_; } else { - return regionBuilder_.getMessage(); + return exceptionBuilder_.getMessage(); } } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public Builder setRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { - if (regionBuilder_ == null) { + public Builder setException(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { + if (exceptionBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - region_ = value; + exception_ = value; onChanged(); } else { - regionBuilder_.setMessage(value); + exceptionBuilder_.setMessage(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public Builder setRegion( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) { - if (regionBuilder_ == null) { - region_ = builderForValue.build(); + public Builder setException( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) { + if (exceptionBuilder_ == null) { + exception_ = builderForValue.build(); onChanged(); } else { - regionBuilder_.setMessage(builderForValue.build()); + exceptionBuilder_.setMessage(builderForValue.build()); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - 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(); + 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 { - region_ = value; + exception_ = value; } onChanged(); } else { - regionBuilder_.mergeFrom(value); + exceptionBuilder_.mergeFrom(value); } - bitField0_ |= 0x00000001; + bitField0_ |= 0x00000002; return this; } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public Builder clearRegion() { - if (regionBuilder_ == null) { - region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + public Builder clearException() { + if (exceptionBuilder_ == null) { + exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); onChanged(); } else { - regionBuilder_.clear(); + exceptionBuilder_.clear(); } - bitField0_ = (bitField0_ & ~0x00000001); + bitField0_ = (bitField0_ & ~0x00000002); return this; } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() { - bitField0_ |= 0x00000001; + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getExceptionBuilder() { + bitField0_ |= 0x00000002; onChanged(); - return getRegionFieldBuilder().getBuilder(); + return getExceptionFieldBuilder().getBuilder(); } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { - if (regionBuilder_ != null) { - return regionBuilder_.getMessageOrBuilder(); + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder() { + if (exceptionBuilder_ != null) { + return exceptionBuilder_.getMessageOrBuilder(); } else { - return region_; + return exception_; } } /** - * required .RegionSpecifier region = 1; + * optional .NameBytesPair exception = 2; + * + *
+       * If the operation failed globally for this region, this exception is set
+       * 
*/ 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_, + 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()); - region_ = null; + exception_ = null; } - return regionBuilder_; + return exceptionBuilder_; } - // repeated .MultiAction action = 2; - private java.util.List action_ = + // @@protoc_insertion_point(builder_scope:RegionActionResult) + } + + static { + defaultInstance = new RegionActionResult(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RegionActionResult) + } + + public interface MultiRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .RegionAction regionAction = 1; + /** + * repeated .RegionAction regionAction = 1; + */ + java.util.List + getRegionActionList(); + /** + * repeated .RegionAction regionAction = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction getRegionAction(int index); + /** + * repeated .RegionAction regionAction = 1; + */ + int getRegionActionCount(); + /** + * repeated .RegionAction regionAction = 1; + */ + java.util.List + getRegionActionOrBuilderList(); + /** + * repeated .RegionAction regionAction = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder( + int index); + } + /** + * Protobuf type {@code MultiRequest} + * + *
+   **
+   * Execute a list of actions on a given region in order.
+   * Nothing prevents a request to contains a set of RegionAction on the same region.
+   * For this reason, the matching between the MultiRequest and the MultiResponse is not
+   *  done by the region specifier but by keeping the order of the RegionActionResult vs.
+   *  the order of the RegionAction.
+   * 
+ */ + public static final class MultiRequest extends + com.google.protobuf.GeneratedMessage + implements MultiRequestOrBuilder { + // Use MultiRequest.newBuilder() to construct. + private MultiRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private MultiRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final MultiRequest defaultInstance; + public static MultiRequest getDefaultInstance() { + return defaultInstance; + } + + public MultiRequest getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private MultiRequest( + 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)) { + regionAction_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + regionAction_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.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)) { + regionAction_ = java.util.Collections.unmodifiableList(regionAction_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public MultiRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new MultiRequest(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + // repeated .RegionAction regionAction = 1; + public static final int REGIONACTION_FIELD_NUMBER = 1; + private java.util.List regionAction_; + /** + * repeated .RegionAction regionAction = 1; + */ + public java.util.List getRegionActionList() { + return regionAction_; + } + /** + * repeated .RegionAction regionAction = 1; + */ + public java.util.List + getRegionActionOrBuilderList() { + return regionAction_; + } + /** + * repeated .RegionAction regionAction = 1; + */ + public int getRegionActionCount() { + return regionAction_.size(); + } + /** + * repeated .RegionAction regionAction = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction getRegionAction(int index) { + return regionAction_.get(index); + } + /** + * repeated .RegionAction regionAction = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder( + int index) { + return regionAction_.get(index); + } + + 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; + + for (int i = 0; i < getRegionActionCount(); i++) { + if (!getRegionAction(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 < regionAction_.size(); i++) { + output.writeMessage(1, regionAction_.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 < regionAction_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, regionAction_.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.MultiRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest) obj; + + boolean result = true; + result = result && getRegionActionList() + .equals(other.getRegionActionList()); + 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 (getRegionActionCount() > 0) { + hash = (37 * hash) + REGIONACTION_FIELD_NUMBER; + hash = (53 * hash) + getRegionActionList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest 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.MultiRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest 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.MultiRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest 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.MultiRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest 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.MultiRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest 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.MultiRequest 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 MultiRequest} + * + *
+     **
+     * Execute a list of actions on a given region in order.
+     * Nothing prevents a request to contains a set of RegionAction on the same region.
+     * For this reason, the matching between the MultiRequest and the MultiResponse is not
+     *  done by the region specifier but by keeping the order of the RegionActionResult vs.
+     *  the order of the RegionAction.
+     * 
+ */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_MultiRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRegionActionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (regionActionBuilder_ == null) { + regionAction_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + regionActionBuilder_.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_MultiRequest_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + 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_; + if (regionActionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + regionAction_ = java.util.Collections.unmodifiableList(regionAction_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.regionAction_ = regionAction_; + } else { + result.regionAction_ = regionActionBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + 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 (regionActionBuilder_ == null) { + if (!other.regionAction_.isEmpty()) { + if (regionAction_.isEmpty()) { + regionAction_ = other.regionAction_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureRegionActionIsMutable(); + regionAction_.addAll(other.regionAction_); + } + onChanged(); + } + } else { + if (!other.regionAction_.isEmpty()) { + if (regionActionBuilder_.isEmpty()) { + regionActionBuilder_.dispose(); + regionActionBuilder_ = null; + regionAction_ = other.regionAction_; + bitField0_ = (bitField0_ & ~0x00000001); + regionActionBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getRegionActionFieldBuilder() : null; + } else { + regionActionBuilder_.addAllMessages(other.regionAction_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean 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_; + + // 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_; + return regionActionBuilder_; } - // 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; - } - // @@protoc_insertion_point(builder_scope:MultiRequest) } @@ -26186,29 +26072,29 @@ 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); } /** @@ -26264,10 +26150,10 @@ } 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; } } @@ -26279,7 +26165,7 @@ 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(); @@ -26312,52 +26198,52 @@ 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; } @@ -26369,8 +26255,8 @@ 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); } @@ -26381,9 +26267,9 @@ 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; @@ -26408,8 +26294,8 @@ 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; @@ -26423,9 +26309,9 @@ } 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; @@ -26528,7 +26414,7 @@ } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getResultFieldBuilder(); + getRegionActionResultFieldBuilder(); } } private static Builder create() { @@ -26537,11 +26423,11 @@ 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; } @@ -26570,14 +26456,14 @@ 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; @@ -26594,29 +26480,29 @@ 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_); } } } @@ -26625,8 +26511,8 @@ } 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; } @@ -26653,244 +26539,244 @@ } 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) @@ -26921,14 +26807,6 @@ com.google.protobuf.RpcCallback done); /** - * rpc MultiGet(.MultiGetRequest) returns (.MultiGetResponse); - */ - public abstract void multiGet( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest request, - com.google.protobuf.RpcCallback done); - - /** * rpc Mutate(.MutateRequest) returns (.MutateResponse); */ public abstract void mutate( @@ -26982,14 +26860,6 @@ } @java.lang.Override - public void multiGet( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest request, - com.google.protobuf.RpcCallback done) { - impl.multiGet(controller, request, done); - } - - @java.lang.Override public void mutate( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest request, @@ -27054,16 +26924,14 @@ case 0: return impl.get(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest)request); case 1: - return impl.multiGet(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest)request); + return impl.mutate(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest)request); case 2: - return impl.mutate(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest)request); + return impl.scan(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest)request); case 3: - return impl.scan(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest)request); + return impl.bulkLoadHFile(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest)request); case 4: - return impl.bulkLoadHFile(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest)request); + return impl.execService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request); case 5: - return impl.execService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request); - case 6: return impl.multi(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request); default: throw new java.lang.AssertionError("Can't get here."); @@ -27082,16 +26950,14 @@ case 0: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest.getDefaultInstance(); case 1: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest.getDefaultInstance(); case 2: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest.getDefaultInstance(); case 3: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); - case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -27110,16 +26976,14 @@ case 0: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse.getDefaultInstance(); case 1: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance(); case 2: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance(); case 3: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); - case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -27138,14 +27002,6 @@ com.google.protobuf.RpcCallback done); /** - * rpc MultiGet(.MultiGetRequest) returns (.MultiGetResponse); - */ - public abstract void multiGet( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest request, - com.google.protobuf.RpcCallback done); - - /** * rpc Mutate(.MutateRequest) returns (.MutateResponse); */ public abstract void mutate( @@ -27213,31 +27069,26 @@ done)); return; case 1: - this.multiGet(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest)request, - com.google.protobuf.RpcUtil.specializeCallback( - done)); - return; - case 2: this.mutate(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 3: + case 2: this.scan(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 4: + case 3: this.bulkLoadHFile(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 5: + case 4: this.execService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 6: + case 5: this.multi(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); @@ -27259,16 +27110,14 @@ case 0: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest.getDefaultInstance(); case 1: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest.getDefaultInstance(); case 2: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest.getDefaultInstance(); case 3: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); - case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -27287,16 +27136,14 @@ case 0: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse.getDefaultInstance(); case 1: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance(); case 2: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance(); case 3: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); - case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -27334,27 +27181,12 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse.getDefaultInstance())); } - public void multiGet( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest request, - com.google.protobuf.RpcCallback done) { - channel.callMethod( - getDescriptor().getMethods().get(1), - controller, - request, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance(), - com.google.protobuf.RpcUtil.generalizeCallback( - done, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.class, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance())); - } - public void mutate( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(2), + getDescriptor().getMethods().get(1), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance(), @@ -27369,7 +27201,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(3), + getDescriptor().getMethods().get(2), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance(), @@ -27384,7 +27216,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(4), + getDescriptor().getMethods().get(3), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance(), @@ -27399,7 +27231,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(5), + getDescriptor().getMethods().get(4), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(), @@ -27414,7 +27246,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(6), + getDescriptor().getMethods().get(5), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(), @@ -27436,11 +27268,6 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest request) throws com.google.protobuf.ServiceException; - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse multiGet( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest request) - throws com.google.protobuf.ServiceException; - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse mutate( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest request) @@ -27486,24 +27313,12 @@ } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse multiGet( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest request) - throws com.google.protobuf.ServiceException { - return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(1), - controller, - request, - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse.getDefaultInstance()); - } - - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse mutate( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(2), + getDescriptor().getMethods().get(1), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse.getDefaultInstance()); @@ -27515,7 +27330,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(3), + getDescriptor().getMethods().get(2), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse.getDefaultInstance()); @@ -27527,7 +27342,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(4), + getDescriptor().getMethods().get(3), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse.getDefaultInstance()); @@ -27539,7 +27354,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(5), + getDescriptor().getMethods().get(4), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()); @@ -27551,7 +27366,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(6), + getDescriptor().getMethods().get(5), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance()); @@ -27583,21 +27398,11 @@ com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_GetRequest_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_MultiGetRequest_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_MultiGetRequest_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor internal_static_GetResponse_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_GetResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_MultiGetResponse_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_MultiGetResponse_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor internal_static_Condition_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -27673,16 +27478,26 @@ 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_MultiAction_fieldAccessorTable; + internal_static_Action_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_ActionResult_descriptor; + internal_static_RegionAction_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_ActionResult_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_ResultOrException_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RegionActionResult_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RegionActionResult_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_MultiRequest_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -27703,98 +27518,98 @@ java.lang.String[] descriptorData = { "\n\014Client.proto\032\013HBase.proto\032\014Filter.prot" + "o\032\nCell.proto\032\020Comparator.proto\"+\n\006Colum" + - "n\022\016\n\006family\030\001 \002(\014\022\021\n\tqualifier\030\002 \003(\014\"\347\001\n" + + "n\022\016\n\006family\030\001 \002(\014\022\021\n\tqualifier\030\002 \003(\014\"\251\002\n" + "\003Get\022\013\n\003row\030\001 \002(\014\022\027\n\006column\030\002 \003(\0132\007.Colu" + "mn\022!\n\tattribute\030\003 \003(\0132\016.NameBytesPair\022\027\n" + "\006filter\030\004 \001(\0132\007.Filter\022\036\n\ntime_range\030\005 \001" + "(\0132\n.TimeRange\022\027\n\014max_versions\030\006 \001(\r:\0011\022" + "\032\n\014cache_blocks\030\007 \001(\010:\004true\022\023\n\013store_lim" + - "it\030\010 \001(\r\022\024\n\014store_offset\030\t \001(\r\"<\n\006Result" + - "\022\023\n\004cell\030\001 \003(\0132\005.Cell\022\035\n\025associated_cell", - "_count\030\002 \001(\005\"u\n\nGetRequest\022 \n\006region\030\001 \002" + - "(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \002(\0132\004.Get\022" + - "\032\n\022closest_row_before\030\003 \001(\010\022\026\n\016existence" + - "_only\030\004 \001(\010\"z\n\017MultiGetRequest\022 \n\006region" + - "\030\001 \002(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \003(\0132\004." + - "Get\022\032\n\022closest_row_before\030\003 \001(\010\022\026\n\016exist" + - "ence_only\030\004 \001(\010\"6\n\013GetResponse\022\027\n\006result" + - "\030\001 \001(\0132\007.Result\022\016\n\006exists\030\002 \001(\010\";\n\020Multi" + - "GetResponse\022\027\n\006result\030\001 \003(\0132\007.Result\022\016\n\006" + - "exists\030\002 \003(\010\"\200\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022", - "\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\"\n\014co" + - "mpare_type\030\004 \002(\0162\014.CompareType\022\037\n\ncompar" + - "ator\030\005 \002(\0132\013.Comparator\"\227\006\n\rMutationProt" + - "o\022\013\n\003row\030\001 \001(\014\0220\n\013mutate_type\030\002 \001(\0162\033.Mu" + - "tationProto.MutationType\0220\n\014column_value" + - "\030\003 \003(\0132\032.MutationProto.ColumnValue\022\021\n\tti" + - "mestamp\030\004 \001(\004\022!\n\tattribute\030\005 \003(\0132\016.NameB" + - "ytesPair\022:\n\ndurability\030\006 \001(\0162\031.MutationP" + - "roto.Durability:\013USE_DEFAULT\022\036\n\ntime_ran" + - "ge\030\007 \001(\0132\n.TimeRange\022\035\n\025associated_cell_", - "count\030\010 \001(\005\032\330\001\n\013ColumnValue\022\016\n\006family\030\001 " + - "\002(\014\022B\n\017qualifier_value\030\002 \003(\0132).MutationP" + - "roto.ColumnValue.QualifierValue\032u\n\016Quali" + - "fierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 " + - "\001(\014\022\021\n\ttimestamp\030\003 \001(\004\022.\n\013delete_type\030\004 " + - "\001(\0162\031.MutationProto.DeleteType\"W\n\nDurabi" + - "lity\022\017\n\013USE_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022\r\n\tA" + - "SYNC_WAL\020\002\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WAL\020\004\"" + - ">\n\014MutationType\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT" + - "\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE\020\003\"p\n\nDeleteType\022\026\n", - "\022DELETE_ONE_VERSION\020\000\022\034\n\030DELETE_MULTIPLE" + - "_VERSIONS\020\001\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DELETE" + - "_FAMILY_VERSION\020\003\"r\n\rMutateRequest\022 \n\006re" + - "gion\030\001 \002(\0132\020.RegionSpecifier\022 \n\010mutation" + - "\030\002 \002(\0132\016.MutationProto\022\035\n\tcondition\030\003 \001(" + - "\0132\n.Condition\"<\n\016MutateResponse\022\027\n\006resul" + - "t\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\344\002\n\004" + - "Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattribu" + - "te\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_row\030\003 " + - "\001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007.", - "Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRange\022\027" + - "\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blocks\030" + - "\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_r" + - "esult_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024\n" + - "\014store_offset\030\014 \001(\r\022&\n\036load_column_famil" + - "ies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\"\236\001\n\013S" + - "canRequest\022 \n\006region\030\001 \001(\0132\020.RegionSpeci" + - "fier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscanner_id\030" + - "\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclose_s" + - "canner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\"y\n\014S", - "canResponse\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n" + - "\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022" + - "\013\n\003ttl\030\004 \001(\r\022\030\n\007results\030\005 \003(\0132\007.Result\"\263" + - "\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 \002(\0132" + - "\020.RegionSpecifier\0225\n\013family_path\030\002 \003(\0132 " + - ".BulkLoadHFileRequest.FamilyPath\022\026\n\016assi" + - "gn_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family" + - "\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRes" + - "ponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorServ" + - "iceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002", - "(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014" + - "\"d\n\031CoprocessorServiceRequest\022 \n\006region\030" + - "\001 \002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027." + - "CoprocessorServiceCall\"]\n\032CoprocessorSer" + - "viceResponse\022 \n\006region\030\001 \002(\0132\020.RegionSpe" + - "cifier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPair\"B\n" + - "\013MultiAction\022 \n\010mutation\030\001 \001(\0132\016.Mutatio" + - "nProto\022\021\n\003get\030\002 \001(\0132\004.Get\"I\n\014ActionResul" + - "t\022\026\n\005value\030\001 \001(\0132\007.Result\022!\n\texception\030\002" + - " \001(\0132\016.NameBytesPair\"^\n\014MultiRequest\022 \n\006", - "region\030\001 \002(\0132\020.RegionSpecifier\022\034\n\006action" + - "\030\002 \003(\0132\014.MultiAction\022\016\n\006atomic\030\003 \001(\010\".\n\r" + - "MultiResponse\022\035\n\006result\030\001 \003(\0132\r.ActionRe" + - "sult2\342\002\n\rClientService\022 \n\003Get\022\013.GetReque" + - "st\032\014.GetResponse\022/\n\010MultiGet\022\020.MultiGetR" + - "equest\032\021.MultiGetResponse\022)\n\006Mutate\022\016.Mu" + - "tateRequest\032\017.MutateResponse\022#\n\004Scan\022\014.S" + - "canRequest\032\r.ScanResponse\022>\n\rBulkLoadHFi" + - "le\022\025.BulkLoadHFileRequest\032\026.BulkLoadHFil" + - "eResponse\022F\n\013ExecService\022\032.CoprocessorSe", - "rviceRequest\032\033.CoprocessorServiceRespons" + - "e\022&\n\005Multi\022\r.MultiRequest\032\016.MultiRespons" + - "eBB\n*org.apache.hadoop.hbase.protobuf.ge" + - "neratedB\014ClientProtosH\001\210\001\001\240\001\001" + "it\030\010 \001(\r\022\024\n\014store_offset\030\t \001(\r\022\035\n\016existe" + + "nce_only\030\n \001(\010:\005false\022!\n\022closest_row_bef", + "ore\030\013 \001(\010:\005false\"L\n\006Result\022\023\n\004cell\030\001 \003(\013" + + "2\005.Cell\022\035\n\025associated_cell_count\030\002 \001(\005\022\016" + + "\n\006exists\030\003 \001(\010\"A\n\nGetRequest\022 \n\006region\030\001" + + " \002(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \002(\0132\004.Ge" + + "t\"&\n\013GetResponse\022\027\n\006result\030\001 \001(\0132\007.Resul" + + "t\"\200\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002" + + " \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\"\n\014compare_type\030" + + "\004 \002(\0162\014.CompareType\022\037\n\ncomparator\030\005 \002(\0132" + + "\013.Comparator\"\227\006\n\rMutationProto\022\013\n\003row\030\001 " + + "\001(\014\0220\n\013mutate_type\030\002 \001(\0162\033.MutationProto", + ".MutationType\0220\n\014column_value\030\003 \003(\0132\032.Mu" + + "tationProto.ColumnValue\022\021\n\ttimestamp\030\004 \001" + + "(\004\022!\n\tattribute\030\005 \003(\0132\016.NameBytesPair\022:\n" + + "\ndurability\030\006 \001(\0162\031.MutationProto.Durabi" + + "lity:\013USE_DEFAULT\022\036\n\ntime_range\030\007 \001(\0132\n." + + "TimeRange\022\035\n\025associated_cell_count\030\010 \001(\005" + + "\032\330\001\n\013ColumnValue\022\016\n\006family\030\001 \002(\014\022B\n\017qual" + + "ifier_value\030\002 \003(\0132).MutationProto.Column" + + "Value.QualifierValue\032u\n\016QualifierValue\022\021" + + "\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\ttime", + "stamp\030\003 \001(\004\022.\n\013delete_type\030\004 \001(\0162\031.Mutat" + + "ionProto.DeleteType\"W\n\nDurability\022\017\n\013USE" + + "_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002\022" + + "\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014Mutation" + + "Type\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002" + + "\022\n\n\006DELETE\020\003\"p\n\nDeleteType\022\026\n\022DELETE_ONE" + + "_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001" + + "\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DELETE_FAMILY_VER" + + "SION\020\003\"r\n\rMutateRequest\022 \n\006region\030\001 \002(\0132" + + "\020.RegionSpecifier\022 \n\010mutation\030\002 \002(\0132\016.Mu", + "tationProto\022\035\n\tcondition\030\003 \001(\0132\n.Conditi" + + "on\"<\n\016MutateResponse\022\027\n\006result\030\001 \001(\0132\007.R" + + "esult\022\021\n\tprocessed\030\002 \001(\010\"\344\002\n\004Scan\022\027\n\006col" + + "umn\030\001 \003(\0132\007.Column\022!\n\tattribute\030\002 \003(\0132\016." + + "NameBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop" + + "_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007.Filter\022\036\n\nt" + + "ime_range\030\006 \001(\0132\n.TimeRange\022\027\n\014max_versi" + + "ons\030\007 \001(\r:\0011\022\032\n\014cache_blocks\030\010 \001(\010:\004true" + + "\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_result_size\030" + + "\n \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024\n\014store_offs", + "et\030\014 \001(\r\022&\n\036load_column_families_on_dema" + + "nd\030\r \001(\010\022\r\n\005small\030\016 \001(\010\"\236\001\n\013ScanRequest\022" + + " \n\006region\030\001 \001(\0132\020.RegionSpecifier\022\023\n\004sca" + + "n\030\002 \001(\0132\005.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016nu" + + "mber_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(" + + "\010\022\025\n\rnext_call_seq\030\006 \001(\004\"y\n\014ScanResponse" + + "\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_id" + + "\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(" + + "\r\022\030\n\007results\030\005 \003(\0132\007.Result\"\263\001\n\024BulkLoad" + + "HFileRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpe", + "cifier\0225\n\013family_path\030\002 \003(\0132 .BulkLoadHF" + + "ileRequest.FamilyPath\022\026\n\016assign_seq_num\030" + + "\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004p" + + "ath\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006lo" + + "aded\030\001 \002(\010\"a\n\026CoprocessorServiceCall\022\013\n\003" + + "row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013metho" + + "d_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"d\n\031Coproce" + + "ssorServiceRequest\022 \n\006region\030\001 \002(\0132\020.Reg" + + "ionSpecifier\022%\n\004call\030\002 \002(\0132\027.Coprocessor" + + "ServiceCall\"]\n\032CoprocessorServiceRespons", + "e\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005v" + + "alue\030\002 \002(\0132\016.NameBytesPair\"L\n\006Action\022\r\n\005" + + "index\030\001 \001(\r\022 \n\010mutation\030\002 \001(\0132\016.Mutation" + + "Proto\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\006at" + + "omic\030\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007.Action\"^\n\021R" + + "esultOrException\022\r\n\005index\030\001 \001(\r\022\027\n\006resul" + + "t\030\002 \001(\0132\007.Result\022!\n\texception\030\003 \001(\0132\016.Na" + + "meBytesPair\"f\n\022RegionActionResult\022-\n\021res" + + "ultOrException\030\001 \003(\0132\022.ResultOrException", + "\022!\n\texception\030\002 \001(\0132\016.NameBytesPair\"3\n\014M" + + "ultiRequest\022#\n\014regionAction\030\001 \003(\0132\r.Regi" + + "onAction\"@\n\rMultiResponse\022/\n\022regionActio" + + "nResult\030\001 \003(\0132\023.RegionActionResult2\261\002\n\rC" + + "lientService\022 \n\003Get\022\013.GetRequest\032\014.GetRe" + + "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() { @@ -27812,45 +27627,33 @@ internal_static_Get_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Get_descriptor, - new java.lang.String[] { "Row", "Column", "Attribute", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "StoreLimit", "StoreOffset", }); + new java.lang.String[] { "Row", "Column", "Attribute", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "StoreLimit", "StoreOffset", "ExistenceOnly", "ClosestRowBefore", }); internal_static_Result_descriptor = getDescriptor().getMessageTypes().get(2); internal_static_Result_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Result_descriptor, - new java.lang.String[] { "Cell", "AssociatedCellCount", }); + new java.lang.String[] { "Cell", "AssociatedCellCount", "Exists", }); internal_static_GetRequest_descriptor = getDescriptor().getMessageTypes().get(3); internal_static_GetRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_GetRequest_descriptor, - new java.lang.String[] { "Region", "Get", "ClosestRowBefore", "ExistenceOnly", }); - internal_static_MultiGetRequest_descriptor = + new java.lang.String[] { "Region", "Get", }); + internal_static_GetResponse_descriptor = getDescriptor().getMessageTypes().get(4); - internal_static_MultiGetRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_MultiGetRequest_descriptor, - new java.lang.String[] { "Region", "Get", "ClosestRowBefore", "ExistenceOnly", }); - internal_static_GetResponse_descriptor = - getDescriptor().getMessageTypes().get(5); internal_static_GetResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_GetResponse_descriptor, - new java.lang.String[] { "Result", "Exists", }); - internal_static_MultiGetResponse_descriptor = - getDescriptor().getMessageTypes().get(6); - internal_static_MultiGetResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_MultiGetResponse_descriptor, - new java.lang.String[] { "Result", "Exists", }); + new java.lang.String[] { "Result", }); internal_static_Condition_descriptor = - getDescriptor().getMessageTypes().get(7); + getDescriptor().getMessageTypes().get(5); internal_static_Condition_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Condition_descriptor, new java.lang.String[] { "Row", "Family", "Qualifier", "CompareType", "Comparator", }); internal_static_MutationProto_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(6); internal_static_MutationProto_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MutationProto_descriptor, @@ -27868,37 +27671,37 @@ internal_static_MutationProto_ColumnValue_QualifierValue_descriptor, new java.lang.String[] { "Qualifier", "Value", "Timestamp", "DeleteType", }); internal_static_MutateRequest_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(7); internal_static_MutateRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MutateRequest_descriptor, new java.lang.String[] { "Region", "Mutation", "Condition", }); internal_static_MutateResponse_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(8); internal_static_MutateResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MutateResponse_descriptor, new java.lang.String[] { "Result", "Processed", }); internal_static_Scan_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(9); internal_static_Scan_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Scan_descriptor, new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", }); internal_static_ScanRequest_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(10); internal_static_ScanRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ScanRequest_descriptor, new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", }); internal_static_ScanResponse_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(11); internal_static_ScanResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ScanResponse_descriptor, new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", }); internal_static_BulkLoadHFileRequest_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(12); internal_static_BulkLoadHFileRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BulkLoadHFileRequest_descriptor, @@ -27910,53 +27713,65 @@ internal_static_BulkLoadHFileRequest_FamilyPath_descriptor, new java.lang.String[] { "Family", "Path", }); internal_static_BulkLoadHFileResponse_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(13); internal_static_BulkLoadHFileResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BulkLoadHFileResponse_descriptor, new java.lang.String[] { "Loaded", }); internal_static_CoprocessorServiceCall_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(14); internal_static_CoprocessorServiceCall_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CoprocessorServiceCall_descriptor, new java.lang.String[] { "Row", "ServiceName", "MethodName", "Request", }); internal_static_CoprocessorServiceRequest_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(15); internal_static_CoprocessorServiceRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CoprocessorServiceRequest_descriptor, new java.lang.String[] { "Region", "Call", }); internal_static_CoprocessorServiceResponse_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(16); internal_static_CoprocessorServiceResponse_fieldAccessorTable = new 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(17); + internal_static_Action_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Action_descriptor, + new java.lang.String[] { "Index", "Mutation", "Get", }); + internal_static_RegionAction_descriptor = + getDescriptor().getMessageTypes().get(18); + internal_static_RegionAction_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RegionAction_descriptor, + new java.lang.String[] { "Region", "Atomic", "Action", }); + internal_static_ResultOrException_descriptor = getDescriptor().getMessageTypes().get(19); - internal_static_MultiAction_fieldAccessorTable = new + internal_static_ResultOrException_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_MultiAction_descriptor, - new java.lang.String[] { "Mutation", "Get", }); - internal_static_ActionResult_descriptor = + internal_static_ResultOrException_descriptor, + new java.lang.String[] { "Index", "Result", "Exception", }); + internal_static_RegionActionResult_descriptor = getDescriptor().getMessageTypes().get(20); - internal_static_ActionResult_fieldAccessorTable = new + internal_static_RegionActionResult_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_ActionResult_descriptor, - new java.lang.String[] { "Value", "Exception", }); + internal_static_RegionActionResult_descriptor, + new java.lang.String[] { "ResultOrException", "Exception", }); internal_static_MultiRequest_descriptor = getDescriptor().getMessageTypes().get(21); 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); 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; } }; Index: hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java =================================================================== --- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java (revision 1529348) +++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RPCProtos.java (working copy) @@ -3662,6 +3662,26 @@ * */ 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 @@ bitField0_ |= 0x00000010; break; } + case 48: { + bitField0_ |= 0x00000020; + priority_ = input.readUInt32(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -3946,12 +3971,39 @@ 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 @@ if (((bitField0_ & 0x00000010) == 0x00000010)) { output.writeMessage(5, cellBlockMeta_); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeUInt32(6, priority_); + } getUnknownFields().writeTo(output); } @@ -4009,6 +4064,10 @@ 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 @@ 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 @@ 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 @@ cellBlockMetaBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000010); + priority_ = 0; + bitField0_ = (bitField0_ & ~0x00000020); return this; } @@ -4279,6 +4349,10 @@ } 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 @@ if (other.hasCellBlockMeta()) { mergeCellBlockMeta(other.getCellBlockMeta()); } + if (other.hasPriority()) { + setPriority(other.getPriority()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -4781,6 +4858,59 @@ 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 @@ "\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 @@ 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 Index: hbase-protocol/src/main/protobuf/Client.proto =================================================================== --- hbase-protocol/src/main/protobuf/Client.proto (revision 1529348) +++ hbase-protocol/src/main/protobuf/Client.proto (working copy) @@ -38,7 +38,10 @@ } /** - * The protocol buffer version of Get + * The protocol buffer version of Get. + * Unless existence_only is specified, return all the requested data + * for the row that matches exactly, or the one that immediately + * precedes it if closest_row_before is specified. */ message Get { required bytes row = 1; @@ -50,6 +53,14 @@ optional bool cache_blocks = 7 [default = true]; optional uint32 store_limit = 8; optional uint32 store_offset = 9; + + // The result isn't asked for, just check for + // the existence. + optional bool existence_only = 10 [default = false]; + + // If the row to get doesn't exist, return the + // closest row before. + optional bool closest_row_before = 11 [default = false]; } message Result { @@ -63,58 +74,24 @@ // ours. NOTE: This is different from the pb managed cell_count of the // 'cell' field above which is non-null when the cells are pb'd. optional int32 associated_cell_count = 2; + + // used for Get to check existence only. Not set if existence_only was not set to true + // in the query. + optional bool exists = 3; } /** * The get request. Perform a single Get operation. - * Unless existence_only is specified, return all the requested data - * for the row that matches exactly, or the one that immediately - * precedes it if closest_row_before is specified. - * - * If existence_only is set, only the existence will be returned. */ message GetRequest { required RegionSpecifier region = 1; required Get get = 2; - - // If the row to get doesn't exist, return the - // closest row before. - optional bool closest_row_before = 3; - - // The result isn't asked for, just check for - // the existence. If closest_row_before specified, - // this will be ignored - optional bool existence_only = 4; } -message MultiGetRequest { - required RegionSpecifier region = 1; - repeated Get get = 2; - - // If the row to get doesn't exist, return the - // closest row before. - optional bool closest_row_before = 3; - - // The result isn't asked for, just check for - // the existence. If closest_row_before specified, - // this will be ignored - optional bool existence_only = 4; -} - message GetResponse { optional Result result = 1; - - // used for Get to check existence only - optional bool exists = 2; } -message MultiGetResponse { - repeated Result result = 1; - - // used for Get to check existence only - repeated bool exists = 2; -} - /** * Condition to check if the value of a given cell (row, * family, qualifier) matches a value via a given comparator. @@ -317,43 +294,60 @@ 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; +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; +} + +/** + * The result of a RegionAction. + */ +message RegionActionResult { + repeated ResultOrException resultOrException = 1; + // If the operation failed globally for this region, this exception is set optional NameBytesPair exception = 2; } /** - * 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. + * Nothing prevents a request to contains a set of RegionAction on the same region. + * For this reason, the matching between the MultiRequest and the MultiResponse is not + * done by the region specifier but by keeping the order of the RegionActionResult vs. + * the order of the RegionAction. */ 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; } @@ -361,9 +355,6 @@ rpc Get(GetRequest) returns(GetResponse); - rpc MultiGet(MultiGetRequest) - returns(MultiGetResponse); - rpc Mutate(MutateRequest) returns(MutateResponse); Index: hbase-protocol/src/main/protobuf/RPC.proto =================================================================== --- hbase-protocol/src/main/protobuf/RPC.proto (revision 1529348) +++ hbase-protocol/src/main/protobuf/RPC.proto (working copy) @@ -119,7 +119,9 @@ 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 { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 1529348) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -73,16 +73,16 @@ 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; @@ -114,11 +114,9 @@ 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; @@ -4356,7 +4354,7 @@ } } List results = get(get, true); - return Result.create(results); + return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null); } /* Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (revision 1529348) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (working copy) @@ -63,6 +63,7 @@ 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.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -148,7 +149,6 @@ 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; @@ -157,14 +157,15 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest; -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.MultiResponse; 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; @@ -2736,7 +2737,7 @@ Boolean existence = null; Result r = null; - if (request.getClosestRowBefore()) { + if (get.hasClosestRowBefore() && get.getClosestRowBefore()) { if (get.getColumnCount() != 1) { throw new DoNotRetryIOException( "get ClosestRowBefore supports one and only one family now, not " @@ -2747,13 +2748,13 @@ r = region.getClosestRowBefore(row, family); } else { Get clientGet = ProtobufUtil.toGet(get); - if (request.getExistenceOnly() && region.getCoprocessorHost() != null) { + if (get.getExistenceOnly() && region.getCoprocessorHost() != null) { existence = region.getCoprocessorHost().preExists(clientGet); } if (existence == null) { r = region.get(clientGet); - if (request.getExistenceOnly()) { - boolean exists = r != null && !r.isEmpty(); + if (get.getExistenceOnly()) { + boolean exists = r.getExists(); if (region.getCoprocessorHost() != null) { exists = region.getCoprocessorHost().postExists(clientGet, exists); } @@ -2761,9 +2762,10 @@ } } } - if (existence != null) { - builder.setExists(existence.booleanValue()); - } else if (r != null) { + if (existence != null){ + ClientProtos.Result pbr = ProtobufUtil.toResult(existence); + builder.setResult(pbr); + }else if (r != null) { ClientProtos.Result pbr = ProtobufUtil.toResult(r); builder.setResult(pbr); } @@ -2775,62 +2777,6 @@ } } - /** - * Get multi data from a table. - * - * @param controller the RPC controller - * @param request multi-the get request - * @throws ServiceException - */ - @Override - public MultiGetResponse multiGet(final RpcController controller, final MultiGetRequest request) - throws ServiceException { - long before = EnvironmentEdgeManager.currentTimeMillis(); - try { - requestCount.add(request.getGetCount()); - HRegion region = getRegion(request.getRegion()); - MultiGetResponse.Builder builder = MultiGetResponse.newBuilder(); - for (ClientProtos.Get get: request.getGetList()) { - Boolean existence = null; - Result r = null; - if (request.getClosestRowBefore()) { - if (get.getColumnCount() != 1) { - throw new DoNotRetryIOException( - "get ClosestRowBefore supports one and only one family now, not " - + get.getColumnCount() + " families"); - } - byte[] row = get.getRow().toByteArray(); - byte[] family = get.getColumn(0).getFamily().toByteArray(); - r = region.getClosestRowBefore(row, family); - } else { - Get clientGet = ProtobufUtil.toGet(get); - if (request.getExistenceOnly() && region.getCoprocessorHost() != null) { - existence = region.getCoprocessorHost().preExists(clientGet); - } - if (existence == null) { - r = region.get(clientGet); - if (request.getExistenceOnly()) { - boolean exists = r != null && !r.isEmpty(); - if (region.getCoprocessorHost() != null) { - exists = region.getCoprocessorHost().postExists(clientGet, exists); - } - existence = exists; - } - } - } - if (existence != null) { - builder.addExists(existence.booleanValue()); - } else if (r != null) { - builder.addResult(ProtobufUtil.toResult(r)); - } - } - return builder.build(); - } catch (IOException ie) { - throw new ServiceException(ie); - } finally { - metricsRegionServer.updateGet(EnvironmentEdgeManager.currentTimeMillis() - before); - } - } /** * Mutate data in a table. @@ -3279,106 +3225,132 @@ // 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. - controller.setCellScanner(null); + if (controller != null) controller.setCellScanner(null); List cellsToReturn = null; - 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); + MultiResponse.Builder responseBuilder = MultiResponse.newBuilder(); + + for (RegionAction regionAction : request.getRegionActionList()) { + this.requestCount.add(regionAction.getActionCount()); + RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.newBuilder(); + HRegion region; + try { + region = getRegion(regionAction.getRegion()); + } catch (IOException e) { + regionActionResultBuilder.setException(ResponseConverter.buildException(e)); + responseBuilder.addRegionActionResult(regionActionResultBuilder.build()); + continue; // For this region it's a failure. + } + + if (regionAction.hasAtomic() && regionAction.getAtomic()) { + // 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. + try { + mutateRows(region, regionAction.getActionList(), cellScanner); + } catch (IOException e) { + // As it's atomic, we may expect it's a global failure. + regionActionResultBuilder.setException(ResponseConverter.buildException(e)); + } + } else { + // doNonAtomicRegionMutation manages the exception internally + cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner, + regionActionResultBuilder, cellsToReturn); + } + responseBuilder.addRegionActionResult(regionActionResultBuilder.build()); + } + // Load the controller with the Cells to return. + if (cellsToReturn != null && !cellsToReturn.isEmpty() && controller != null) { + controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn)); + } + return responseBuilder.build(); + } + + /** + * 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 actions + * @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()); + r = region.get(get); + } 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"); } - // 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()) { - 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 (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(); + cellsToReturn.add(r); + } else { + pbResult = ProtobufUtil.toResult(r); } + resultOrExceptionBuilder = + ClientProtos.ResultOrException.newBuilder().setResult(pbResult); } - if (!mutations.isEmpty()) { - doBatchOp(builder, region, mutations, cellScanner); - } + // 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)); } - // Load the controller with the Cells to return. - if (cellsToReturn != null && !cellsToReturn.isEmpty()) { - controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn)); + if (resultOrExceptionBuilder != null) { + // Propagate index. + resultOrExceptionBuilder.setIndex(action.getIndex()); + builder.addResultOrException(resultOrExceptionBuilder.build()); } - return builder.build(); - } catch (IOException ie) { - throw new ServiceException(ie); } + // Finish up any outstanding mutations + if (mutations != null && !mutations.isEmpty()) { + doBatchOp(builder, region, mutations, cellScanner); + } + return cellsToReturn; } // End Client methods @@ -3856,20 +3828,19 @@ 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(); List> walEntries = new ArrayList>(); List> mutations = new ArrayList>(); for (WALEntry entry : entries) { - Pair walEntry = (coprocessorHost == null) ? null : + Pair walEntry = (coprocessorHost == null) ? null : new Pair(); - List> edits = HLogSplitter.getMutationsFromWALEntry(entry, + List> edits = HLogSplitter.getMutationsFromWALEntry(entry, cells, walEntry); if (coprocessorHost != null) { // Start coprocessor replay here. The coprocessor is for each WALEdit instead of a @@ -4041,17 +4012,17 @@ * @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); @@ -4061,7 +4032,6 @@ batchContainsDelete = true; } mArray[i++] = mutation; - builder.addResult(result); } requestCount.add(mutations.size()); @@ -4071,33 +4041,33 @@ 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.addResultOrException(resultOrException); } } long after = EnvironmentEdgeManager.currentTimeMillis(); @@ -4108,7 +4078,19 @@ 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 * constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse. @@ -4119,8 +4101,9 @@ * 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; @@ -4154,32 +4137,35 @@ * Mutate a list of rows atomically. * * @param region - * @param mutations + * @param actions * @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); @@ -4382,7 +4368,6 @@ /** * Return the last failed RS name under /hbase/recovering-regions/encodedRegionName * @param encodedRegionName - * @throws IOException * @throws KeeperException */ private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java (revision 1529348) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/QosFunction.java (working copy) @@ -83,8 +83,7 @@ CompactRegionRequest.class, GetRequest.class, MutateRequest.class, - ScanRequest.class, - MultiRequest.class + ScanRequest.class }; // Some caches for helping performance @@ -101,7 +100,7 @@ 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()); @@ -154,6 +153,11 @@ 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; @@ -203,4 +207,4 @@ void setRegionServer(final HRegionServer hrs) { this.hRegionServer = hrs; } -} \ No newline at end of file +} Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java (revision 1529348) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java (working copy) @@ -23,8 +23,6 @@ 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.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.AdminService; 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.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; -import com.google.protobuf.ByteString; import com.google.protobuf.ServiceException; /** @@ -118,7 +104,7 @@ 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 @@ } regionEntries.add(entry); } - + long startTime = EnvironmentEdgeManager.currentTimeMillis(); // replaying edits by region @@ -143,7 +129,7 @@ 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 @@ } } } - + /** * Callable that handles the replay method call going against a single regionserver * @param @@ -202,7 +188,7 @@ this.regionInfo = regionInfo; setLocation(regionLoc); } - + @Override public ReplicateWALEntryResponse call() throws IOException { try { Index: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (working copy) @@ -4356,8 +4356,7 @@ d.deleteColumns(FAMILY, QUALIFIERS[0]); arm.add(d); // TODO: Trying mutateRow again. The batch was failing with a one try only. - // t.mutateRow(arm); - t.batch(Arrays.asList((Row)arm)); + t.mutateRow(arm); r = t.get(g); assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1]))); assertNull(r.getValue(FAMILY, QUALIFIERS[0])); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (working copy) @@ -43,7 +43,6 @@ 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.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 @@ 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()); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java (working copy) @@ -179,7 +179,6 @@ 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 @@ 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 @@ validateEmpty(result); } } - -} - +} \ No newline at end of file Index: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java (working copy) @@ -56,6 +56,7 @@ 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; @@ -73,6 +74,8 @@ import com.google.protobuf.Message; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; /** * Some basic ipc tests. @@ -284,7 +287,7 @@ for (int i = 0; i < cycles; i++) { List cells = new ArrayList(); // Message param = RequestConverter.buildMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm); - Message param = RequestConverter.buildNoDataMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm, cells); + ClientProtos.RegionAction.Builder builder = RequestConverter.buildNoDataRegionAction( CellScanner cellScanner = CellUtil.createCellScanner(cells); if (i % 1000 == 0) { LOG.info("" + i); @@ -293,7 +296,7 @@ // "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()) { Index: hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java (working copy) @@ -78,10 +78,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest; -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.MultiResponse; 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.ScanRequest; @@ -344,22 +341,9 @@ return builder.build(); } - @Override - public MultiGetResponse multiGet(RpcController controller, MultiGetRequest requests) - throws ServiceException { - byte[] regionName = requests.getRegion().getValue().toByteArray(); - Map m = this.gets.get(regionName); - MultiGetResponse.Builder builder = MultiGetResponse.newBuilder(); - if (m != null) { - for (ClientProtos.Get get: requests.getGetList()) { - byte[] row = get.getRow().toByteArray(); - builder.addResult(ProtobufUtil.toResult(m.get(row))); - } - } - return builder.build(); - } + @Override public MutateResponse mutate(RpcController controller, MutateRequest request) throws ServiceException { Index: hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java (working copy) @@ -289,6 +289,9 @@ UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration()); // load the table (creates 4 hfiles) UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM); + UTIL.flush(TABLE_NAME); + // Put some more data into the table so for sure we get more storefiles. + UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM); // disable the table so we can take a snapshot admin.disableTable(TABLE_NAME); @@ -299,7 +302,6 @@ byte[] snapshotNameBytes = Bytes.toBytes(snapshotName); admin.snapshot(snapshotNameBytes, TABLE_NAME); - Configuration conf = master.getConfiguration(); LOG.info("After snapshot File-System state"); FSUtils.logFileSystemState(fs, rootDir, LOG); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionBusyWait.java (working copy) @@ -48,7 +48,7 @@ /** * Test RegionTooBusyException thrown when region is busy */ - @Test (timeout=2000) + @Test (timeout=6000) public void testRegionTooBusy() throws IOException { String method = "testRegionTooBusy"; byte[] tableName = Bytes.toBytes(method); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQosFunction.java (working copy) @@ -20,8 +20,8 @@ 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; Index: hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java (revision 1529348) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/rest/TestResourceFilter.java (working copy) @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; import org.apache.hadoop.hbase.rest.client.Response; @@ -29,7 +29,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -@Category(SmallTests.class) +@Category(MediumTests.class) public class TestResourceFilter { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();