HBASE-9612 Ability to batch edits destined to different regions
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1529348 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
4411592cbb
commit
36e49e01e3
|
@ -255,10 +255,12 @@ class AsyncProcess<CResult> {
|
|||
* @param atLeastOne true if we should submit at least a subset.
|
||||
*/
|
||||
public void submit(List<? extends Row> 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<HRegionLocation, MultiAction<Row>> actionsByServer =
|
||||
new HashMap<HRegionLocation, MultiAction<Row>>();
|
||||
List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
|
||||
|
@ -321,10 +323,7 @@ class AsyncProcess<CResult> {
|
|||
* @return the destination. Null if we couldn't find it.
|
||||
*/
|
||||
private HRegionLocation findDestLocation(Row row, int numAttempt, int posInList) {
|
||||
if (row == null){
|
||||
throw new IllegalArgumentException("row cannot be null");
|
||||
}
|
||||
|
||||
if (row == null) throw new IllegalArgumentException("row cannot be null");
|
||||
HRegionLocation loc = null;
|
||||
IOException locationException = null;
|
||||
try {
|
||||
|
@ -476,29 +475,29 @@ class AsyncProcess<CResult> {
|
|||
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<HRegionLocation, MultiAction<Row>> e : actionsByServer.entrySet()) {
|
||||
final HRegionLocation loc = e.getKey();
|
||||
final MultiAction<Row> multi = e.getValue();
|
||||
incTaskCounters(multi.getRegions(), loc.getServerName());
|
||||
|
||||
final MultiAction<Row> multiAction = e.getValue();
|
||||
incTaskCounters(multiAction.getRegions(), loc.getServerName());
|
||||
Runnable runnable = Trace.wrap("AsyncProcess.sendMultiAction", new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
MultiResponse res;
|
||||
try {
|
||||
MultiServerCallable<Row> callable = createCallable(loc, multi);
|
||||
MultiServerCallable<Row> callable = createCallable(loc, multiAction);
|
||||
try {
|
||||
res = createCaller(callable).callWithoutRetries(callable);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("The call to the region server failed, we don't know where we stand, " +
|
||||
loc.getServerName(), e);
|
||||
resubmitAll(initialActions, multi, loc, numAttempt + 1, e, errorsByServer);
|
||||
LOG.warn("Call to " + loc.getServerName() + " failed numAttempt=" + numAttempt +
|
||||
", resubmitting all since not sure where we are at", e);
|
||||
resubmitAll(initialActions, multiAction, loc, numAttempt + 1, e, errorsByServer);
|
||||
return;
|
||||
}
|
||||
|
||||
receiveMultiAction(initialActions, multi, loc, res, numAttempt, errorsByServer);
|
||||
receiveMultiAction(initialActions, multiAction, loc, res, numAttempt, errorsByServer);
|
||||
} finally {
|
||||
decTaskCounters(multi.getRegions(), loc.getServerName());
|
||||
decTaskCounters(multiAction.getRegions(), loc.getServerName());
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -508,12 +507,12 @@ class AsyncProcess<CResult> {
|
|||
} 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -592,10 +591,9 @@ class AsyncProcess<CResult> {
|
|||
hConnection.updateCachedLocations(tableName,
|
||||
rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location);
|
||||
errorsByServer.reportServerError(location);
|
||||
|
||||
List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
|
||||
for (List<Action<Row>> actions : rsActions.actions.values()) {
|
||||
for (Action<Row> action : actions) {
|
||||
List<Action<Row>> toReplay = new ArrayList<Action<Row>>(initialActions.size());
|
||||
for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
|
||||
for (Action<Row> action : e.getValue()) {
|
||||
if (manageError(numAttempt, action.getOriginalIndex(), action.getAction(),
|
||||
true, t, location)) {
|
||||
toReplay.add(action);
|
||||
|
@ -605,7 +603,7 @@ class AsyncProcess<CResult> {
|
|||
|
||||
if (toReplay.isEmpty()) {
|
||||
LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all " +
|
||||
initialActions.size() + "ops, NOT resubmitting, " + location.getServerName());
|
||||
initialActions.size() + " ops, NOT resubmitting, " + location.getServerName());
|
||||
} else {
|
||||
submit(initialActions, toReplay, numAttempt, errorsByServer);
|
||||
}
|
||||
|
@ -669,11 +667,11 @@ class AsyncProcess<CResult> {
|
|||
}
|
||||
} else { // success
|
||||
if (callback != null) {
|
||||
Action<Row> correspondingAction = initialActions.get(regionResult.getFirst());
|
||||
int index = regionResult.getFirst();
|
||||
Action<Row> correspondingAction = initialActions.get(index);
|
||||
Row row = correspondingAction.getAction();
|
||||
//noinspection unchecked
|
||||
this.callback.success(correspondingAction.getOriginalIndex(),
|
||||
resultsForRS.getKey(), row, (CResult) result);
|
||||
this.callback.success(index, resultsForRS.getKey(), row, (CResult) result);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -694,8 +692,7 @@ class AsyncProcess<CResult> {
|
|||
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 @@ class AsyncProcess<CResult> {
|
|||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -171,6 +171,7 @@ public class ClientSmallScanner extends ClientScanner {
|
|||
ScanResponse response = null;
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
try {
|
||||
controller.setPriority(getTableName());
|
||||
response = getStub().scan(controller, request);
|
||||
return ResponseConverter.getResults(controller.cellScanner(),
|
||||
response);
|
||||
|
|
|
@ -32,7 +32,6 @@ import java.util.TreeSet;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -72,6 +71,8 @@ public class Get extends OperationWithAttributes
|
|||
private int storeOffset = 0;
|
||||
private Filter filter = null;
|
||||
private TimeRange tr = new TimeRange();
|
||||
private boolean checkExistenceOnly = false;
|
||||
private boolean closestRowBefore = false;
|
||||
private Map<byte [], NavigableSet<byte []>> familyMap =
|
||||
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
|
||||
|
||||
|
@ -87,6 +88,22 @@ public class Get extends OperationWithAttributes
|
|||
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.
|
||||
* <p>
|
||||
|
|
|
@ -643,6 +643,7 @@ public class HBaseAdmin implements Abortable, Closeable {
|
|||
.getServerName());
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
|
||||
try {
|
||||
controller.setPriority(tableName);
|
||||
ScanResponse response = server.scan(controller, request);
|
||||
values = ResponseConverter.getResults(controller.cellScanner(), response);
|
||||
} catch (ServiceException se) {
|
||||
|
|
|
@ -2165,8 +2165,7 @@ public class HConnectionManager {
|
|||
boolean isStaleDelete = false;
|
||||
HRegionLocation oldLocation;
|
||||
synchronized (this.cachedRegionLocations) {
|
||||
Map<byte[], HRegionLocation> tableLocations =
|
||||
getTableLocations(hri.getTable());
|
||||
Map<byte[], HRegionLocation> 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.
|
||||
|
@ -2363,6 +2362,7 @@ public class HConnectionManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Return the number of cached region for a table. It will only be called
|
||||
* from a unit test.
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.IOException;
|
|||
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.ProtobufUtil;
|
|||
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 @@ import com.google.protobuf.ServiceException;
|
|||
* <p>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
|
||||
*
|
||||
* <p>To access a table in a multi threaded environment, please consider
|
||||
* using the {@link HTablePool} class to create your HTable instances.
|
||||
*
|
||||
* <p>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 @@ public class HTable implements HTableInterface {
|
|||
new RegionServerCallable<Void>(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 @@ public class HTable implements HTableInterface {
|
|||
MutateRequest request = RequestConverter.buildMutateRequest(
|
||||
getLocation().getRegionInfo().getRegionName(), append);
|
||||
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
|
||||
rpcController.setPriority(getTableName());
|
||||
MutateResponse response = getStub().mutate(rpcController, request);
|
||||
if (!response.hasResult()) return null;
|
||||
return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
|
||||
|
@ -1013,9 +1014,10 @@ public class HTable implements HTableInterface {
|
|||
try {
|
||||
MutateRequest request = RequestConverter.buildMutateRequest(
|
||||
getLocation().getRegionInfo().getRegionName(), increment);
|
||||
PayloadCarryingRpcController rpcContoller = new PayloadCarryingRpcController();
|
||||
MutateResponse response = getStub().mutate(rpcContoller, request);
|
||||
return ProtobufUtil.toResult(response.getResult(), rpcContoller.cellScanner());
|
||||
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
|
||||
rpcController.setPriority(getTableName());
|
||||
MutateResponse response = getStub().mutate(rpcController, request);
|
||||
return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -1074,6 +1076,7 @@ public class HTable implements HTableInterface {
|
|||
getLocation().getRegionInfo().getRegionName(), row, family,
|
||||
qualifier, amount, durability);
|
||||
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
|
||||
rpcController.setPriority(getTableName());
|
||||
MutateResponse response = getStub().mutate(rpcController, request);
|
||||
Result result =
|
||||
ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
|
||||
|
@ -1142,61 +1145,10 @@ public class HTable implements HTableInterface {
|
|||
*/
|
||||
@Override
|
||||
public boolean exists(final Get get) throws IOException {
|
||||
RegionServerCallable<Boolean> callable =
|
||||
new RegionServerCallable<Boolean>(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.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<SortedGet> {
|
||||
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;
|
||||
}
|
||||
get.setCheckExistenceOnly(true);
|
||||
Result r = get(get);
|
||||
assert r.getExists() != null;
|
||||
return r.getExists();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1204,100 +1156,26 @@ public class HTable implements HTableInterface {
|
|||
*/
|
||||
@Override
|
||||
public Boolean[] exists(final List<Get> 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<SortedGet> sortedGetsList = new ArrayList<HTable.SortedGet>();
|
||||
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))};
|
||||
|
||||
for (Get g: gets){
|
||||
g.setCheckExistenceOnly(true);
|
||||
}
|
||||
|
||||
// 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<Integer, List<Get>> getsByRegion = new HashMap<Integer, List<Get>>();
|
||||
|
||||
// Reference map to quickly find back in which region a get belongs.
|
||||
Map<Get, Integer> getToRegionIndexMap = new HashMap<Get, Integer>();
|
||||
Pair<byte[][], byte[][]> 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<Get> regionGets = getsByRegion.get(regionIndex);
|
||||
if (regionGets == null) {
|
||||
regionGets = new ArrayList<Get>();
|
||||
getsByRegion.put(regionIndex, regionGets);
|
||||
}
|
||||
regionGets.add(get.getGet());
|
||||
getToRegionIndexMap.put(get.getGet(), regionIndex);
|
||||
}
|
||||
|
||||
// step 2: make the requests
|
||||
Map<Integer, Future<List<Boolean>>> futures =
|
||||
new HashMap<Integer, Future<List<Boolean>>>(sortedGetsList.size());
|
||||
for (final Map.Entry<Integer, List<Get>> getsByRegionEntry : getsByRegion.entrySet()) {
|
||||
Callable<List<Boolean>> callable = new Callable<List<Boolean>>() {
|
||||
public List<Boolean> call() throws Exception {
|
||||
RegionServerCallable<List<Boolean>> callable =
|
||||
new RegionServerCallable<List<Boolean>>(connection, getName(),
|
||||
getsByRegionEntry.getValue().get(0).getRow()) {
|
||||
public List<Boolean> call() throws IOException {
|
||||
Object[] r1;
|
||||
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.<List<Boolean>> newCaller().callWithRetries(callable,
|
||||
operationTimeout);
|
||||
}
|
||||
};
|
||||
futures.put(getsByRegionEntry.getKey(), pool.submit(callable));
|
||||
}
|
||||
|
||||
// step 3: collect the failures and successes
|
||||
Map<Integer, List<Boolean>> responses = new HashMap<Integer, List<Boolean>>();
|
||||
for (final Map.Entry<Integer, List<Get>> sortedGetEntry : getsByRegion.entrySet()) {
|
||||
try {
|
||||
Future<List<Boolean>> future = futures.get(sortedGetEntry.getKey());
|
||||
List<Boolean> 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());
|
||||
r1 = batch(gets);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Failed for gets on region: " + sortedGetEntry.getKey());
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
Boolean[] results = new Boolean[sortedGetsList.size()];
|
||||
|
||||
// step 4: build the response.
|
||||
Map<Integer, Integer> indexes = new HashMap<Integer, Integer>();
|
||||
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);
|
||||
// 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();
|
||||
}
|
||||
|
||||
return results;
|
||||
|
|
|
@ -24,14 +24,17 @@ import java.util.Map;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -42,91 +45,74 @@ import com.google.protobuf.ServiceException;
|
|||
* @param <R>
|
||||
*/
|
||||
class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
|
||||
private final MultiAction<R> multi;
|
||||
private final MultiAction<R> multiAction;
|
||||
private final boolean cellBlock;
|
||||
|
||||
MultiServerCallable(final HConnection connection, final TableName tableName,
|
||||
final HRegionLocation location, final MultiAction<R> multi) {
|
||||
super(connection, tableName, null);
|
||||
this.multi = multi;
|
||||
this.multiAction = multi;
|
||||
setLocation(location);
|
||||
this.cellBlock = isCellBlock();
|
||||
}
|
||||
|
||||
MultiAction<R> 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<byte[], List<Action<R>>> e: this.multi.actions.entrySet()) {
|
||||
byte[] regionName = e.getKey();
|
||||
int rowMutations = 0;
|
||||
List<Action<R>> actions = e.getValue();
|
||||
for (Action<R> 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;
|
||||
int countOfActions = this.multiAction.size();
|
||||
if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions");
|
||||
MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();
|
||||
List<CellScannable> cells = null;
|
||||
MultiRequest multiRequest;
|
||||
try {
|
||||
// The multi object is a list of Actions by region. Iterate by region.
|
||||
for (Map.Entry<byte[], List<Action<R>>> e: this.multiAction.actions.entrySet()) {
|
||||
final byte [] regionName = e.getKey();
|
||||
final List<Action<R>> actions = e.getValue();
|
||||
RegionAction.Builder regionActionBuilder;
|
||||
if (this.cellBlock) {
|
||||
// Stick all Cells for all RowMutations in here into 'cells'. Populated when we call
|
||||
// buildNoDataMultiRequest in the below.
|
||||
cells = new ArrayList<CellScannable>(rms.getMutations().size());
|
||||
// Build a multi request absent its Cell payload (this is the 'nodata' in the below).
|
||||
multiRequest = RequestConverter.buildNoDataMultiRequest(regionName, rms, cells);
|
||||
} else {
|
||||
multiRequest = RequestConverter.buildMultiRequest(regionName, rms);
|
||||
}
|
||||
// Carry the cells if any over the proxy/pb Service interface using the payload
|
||||
// carrying rpc controller.
|
||||
getStub().multi(new PayloadCarryingRpcController(cells), multiRequest);
|
||||
// This multi call does not return results.
|
||||
response.add(regionName, action.getOriginalIndex(), Result.EMPTY_RESULT);
|
||||
} catch (ServiceException se) {
|
||||
response.add(regionName, action.getOriginalIndex(),
|
||||
ProtobufUtil.getRemoteException(se));
|
||||
}
|
||||
rowMutations++;
|
||||
}
|
||||
}
|
||||
// Are there any non-RowMutation actions to send for this region?
|
||||
if (actions.size() > rowMutations) {
|
||||
Exception ex = null;
|
||||
List<Object> results = null;
|
||||
List<CellScannable> cells = null;
|
||||
MultiRequest multiRequest;
|
||||
try {
|
||||
if (isCellBlock()) {
|
||||
// Presize. Presume at least a KV per Action. There are likely more.
|
||||
if (cells == null) cells = new ArrayList<CellScannable>(countOfActions);
|
||||
// Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
|
||||
// They have already been handled above.
|
||||
cells = new ArrayList<CellScannable>(actions.size() - rowMutations);
|
||||
multiRequest = RequestConverter.buildNoDataMultiRequest(regionName, actions, cells);
|
||||
// They have already been handled above. Guess at count of cells
|
||||
regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells);
|
||||
} else {
|
||||
multiRequest = RequestConverter.buildMultiRequest(regionName, actions);
|
||||
regionActionBuilder = RequestConverter.buildRegionAction(regionName, actions);
|
||||
}
|
||||
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);
|
||||
ClientProtos.MultiResponse responseProto = getStub().multi(controller, multiRequest);
|
||||
results = ResponseConverter.getResults(responseProto, controller.cellScanner());
|
||||
} catch (ServiceException se) {
|
||||
ex = ProtobufUtil.getRemoteException(se);
|
||||
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));
|
||||
}
|
||||
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));
|
||||
}
|
||||
}
|
||||
}
|
||||
return response;
|
||||
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 <code>t</code> -- 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<Integer, Object>(action.getIndex(), t));
|
||||
}
|
||||
}
|
||||
return massFailedResponse;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return True if we should send data in cellblocks. This is an expensive call. Cache the
|
||||
|
|
|
@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
@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 @@ public class Result implements CellScannable {
|
|||
@Deprecated
|
||||
public Result(List<KeyValue> 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,7 +118,14 @@ public class Result implements CellScannable {
|
|||
* @param cells List of cells
|
||||
*/
|
||||
public static Result create(List<Cell> 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<Cell> cells, Boolean exists) {
|
||||
if (exists != null){
|
||||
return new Result(null, exists);
|
||||
}
|
||||
return new Result(cells.toArray(new Cell[cells.size()]), exists);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -126,12 +134,13 @@ public class Result implements CellScannable {
|
|||
* @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 class Result implements CellScannable {
|
|||
public CellScanner cellScanner() {
|
||||
return CellUtil.createCellScanner(this.cells);
|
||||
}
|
||||
|
||||
public Boolean getExists() {
|
||||
return exists;
|
||||
}
|
||||
|
||||
public void setExists(Boolean exists) {
|
||||
this.exists = exists;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -163,6 +163,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
|
|||
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
|
||||
|
|
|
@ -23,7 +23,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
@ -36,6 +37,15 @@ import com.google.protobuf.RpcController;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class PayloadCarryingRpcController implements RpcController, CellScannable {
|
||||
/**
|
||||
* Priority to set on this request. Set it here in controller so available composing the
|
||||
* request. This is the ordained way of setting priorities going forward. We will be
|
||||
* undoing the old annotation-based mechanism.
|
||||
*/
|
||||
// Currently only multi call makes use of this. Eventually this should be only way to set
|
||||
// priority.
|
||||
private int priority = 0;
|
||||
|
||||
// TODO: Fill out the rest of this class methods rather than return UnsupportedOperationException
|
||||
|
||||
/**
|
||||
|
@ -103,4 +113,26 @@ public class PayloadCarryingRpcController implements RpcController, CellScannabl
|
|||
public void startCancel() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param priority Priority for this request; should fall roughly in the range
|
||||
* {@link HConstants#NORMAL_QOS} to {@link HConstants#HIGH_QOS}
|
||||
*/
|
||||
public void setPriority(int priority) {
|
||||
this.priority = priority;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param tn Set priority based off the table we are going against.
|
||||
*/
|
||||
public void setPriority(final TableName tn) {
|
||||
this.priority = tn != null && tn.isSystemTable()? HConstants.HIGH_QOS: HConstants.NORMAL_QOS;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The priority of this request
|
||||
*/
|
||||
public int getPriority() {
|
||||
return priority;
|
||||
}
|
||||
}
|
|
@ -55,6 +55,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
|
@ -211,7 +212,8 @@ public class RpcClient {
|
|||
@SuppressWarnings("serial")
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public static class FailedServerException extends IOException {
|
||||
// Shouldn't this be a DoNotRetryException? St.Ack 10/2/2013
|
||||
public static class FailedServerException extends HBaseIOException {
|
||||
public FailedServerException(String s) {
|
||||
super(s);
|
||||
}
|
||||
|
@ -967,8 +969,12 @@ public class RpcClient {
|
|||
}
|
||||
|
||||
// close the streams and therefore the socket
|
||||
if (this.out != null) {
|
||||
synchronized(this.out) {
|
||||
IOUtils.closeStream(out);
|
||||
this.out = null;
|
||||
}
|
||||
}
|
||||
IOUtils.closeStream(in);
|
||||
this.in = null;
|
||||
disposeSasl();
|
||||
|
@ -1002,9 +1008,10 @@ public class RpcClient {
|
|||
* Note: this is not called from the Connection thread, but by other
|
||||
* threads.
|
||||
* @param call
|
||||
* @param priority
|
||||
* @see #readResponse()
|
||||
*/
|
||||
protected void writeRequest(Call call) {
|
||||
protected void writeRequest(Call call, final int priority) {
|
||||
if (shouldCloseConnection.get()) return;
|
||||
try {
|
||||
RequestHeader.Builder builder = RequestHeader.newBuilder();
|
||||
|
@ -1022,6 +1029,8 @@ public class RpcClient {
|
|||
cellBlockBuilder.setLength(cellBlock.limit());
|
||||
builder.setCellBlockMeta(cellBlockBuilder.build());
|
||||
}
|
||||
// Only pass priority if there one. Let zero be same as no priority.
|
||||
if (priority != 0) builder.setPriority(priority);
|
||||
//noinspection SynchronizeOnNonFinalField
|
||||
RequestHeader header = builder.build();
|
||||
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
|
||||
|
@ -1380,6 +1389,12 @@ public class RpcClient {
|
|||
}
|
||||
}
|
||||
|
||||
Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
|
||||
Message returnType, User ticket, InetSocketAddress addr, int rpcTimeout)
|
||||
throws InterruptedException, IOException {
|
||||
return call(md, param, cells, returnType, ticket, addr, rpcTimeout, HConstants.NORMAL_QOS);
|
||||
}
|
||||
|
||||
/** Make a call, passing <code>param</code>, to the IPC server running at
|
||||
* <code>address</code> which is servicing the <code>protocol</code> protocol,
|
||||
* with the <code>ticket</code> credentials, returning the value.
|
||||
|
@ -1400,12 +1415,12 @@ public class RpcClient {
|
|||
*/
|
||||
Pair<Message, CellScanner> call(MethodDescriptor md, Message param, CellScanner cells,
|
||||
Message returnType, User ticket, InetSocketAddress addr,
|
||||
int rpcTimeout)
|
||||
int rpcTimeout, int priority)
|
||||
throws InterruptedException, IOException {
|
||||
Call call = new Call(md, param, cells, returnType);
|
||||
Connection connection =
|
||||
getConnection(ticket, call, addr, rpcTimeout, this.codec, this.compressor);
|
||||
connection.writeRequest(call); // send the parameter
|
||||
connection.writeRequest(call, priority); // send the parameter
|
||||
boolean interrupted = false;
|
||||
//noinspection SynchronizationOnLocalVariableOrMethodParameter
|
||||
synchronized (call) {
|
||||
|
@ -1632,7 +1647,8 @@ public class RpcClient {
|
|||
}
|
||||
Pair<Message, CellScanner> 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());
|
||||
|
|
|
@ -407,6 +407,12 @@ public final class ProtobufUtil {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (proto.hasExistenceOnly() && proto.getExistenceOnly()){
|
||||
get.setCheckExistenceOnly(true);
|
||||
}
|
||||
if (proto.hasClosestRowBefore() && proto.getClosestRowBefore()){
|
||||
get.setClosestRowBefore(true);
|
||||
}
|
||||
return get;
|
||||
}
|
||||
|
||||
|
@ -922,6 +928,12 @@ public final class ProtobufUtil {
|
|||
if (get.getRowOffsetPerColumnFamily() > 0) {
|
||||
builder.setStoreOffset(get.getRowOffsetPerColumnFamily());
|
||||
}
|
||||
if (get.isCheckExistenceOnly()){
|
||||
builder.setExistenceOnly(true);
|
||||
}
|
||||
if (get.isClosestRowBefore()){
|
||||
builder.setClosestRowBefore(true);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -1053,6 +1065,21 @@ public final class ProtobufUtil {
|
|||
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();
|
||||
}
|
||||
|
||||
|
@ -1066,6 +1093,9 @@ public final class ProtobufUtil {
|
|||
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();
|
||||
}
|
||||
|
||||
|
@ -1076,12 +1106,16 @@ public final class ProtobufUtil {
|
|||
* @return the converted client Result
|
||||
*/
|
||||
public static Result toResult(final ClientProtos.Result proto) {
|
||||
if (proto.hasExists()) {
|
||||
return Result.create(null, proto.getExists());
|
||||
}
|
||||
|
||||
List<CellProtos.Cell> values = proto.getCellList();
|
||||
List<Cell> cells = new ArrayList<Cell>(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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1094,6 +1128,10 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
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<Cell> cells = null;
|
||||
if (proto.hasAssociatedCellCount()) {
|
||||
|
@ -1109,7 +1147,7 @@ public final class ProtobufUtil {
|
|||
for (CellProtos.Cell c: values) {
|
||||
cells.add(toCell(c));
|
||||
}
|
||||
return Result.create(cells);
|
||||
return Result.create(cells, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2257,11 +2295,15 @@ public final class ProtobufUtil {
|
|||
", row=" + getStringForByteString(r.getGet().getRow());
|
||||
} else if (m instanceof ClientProtos.MultiRequest) {
|
||||
ClientProtos.MultiRequest r = (ClientProtos.MultiRequest) m;
|
||||
ClientProtos.MultiAction action = r.getActionList().get(0);
|
||||
return "region= " + getStringForByteString(r.getRegion().getValue()) +
|
||||
", for " + r.getActionCount() +
|
||||
" actions and 1st row key=" + getStringForByteString(action.hasMutation() ?
|
||||
action.getMutation().getRow() : action.getGet().getRow());
|
||||
// Get first set of Actions.
|
||||
ClientProtos.RegionAction actions = r.getRegionActionList().get(0);
|
||||
String row = actions.getActionCount() <= 0? "":
|
||||
getStringForByteString(actions.getAction(0).hasGet()?
|
||||
actions.getAction(0).getGet().getRow():
|
||||
actions.getAction(0).getMutation().getRow());
|
||||
return "region= " + getStringForByteString(actions.getRegion().getValue()) +
|
||||
", for " + r.getRegionActionCount() +
|
||||
" actions and 1st row key=" + row;
|
||||
} else if (m instanceof ClientProtos.MutateRequest) {
|
||||
ClientProtos.MutateRequest r = (ClientProtos.MutateRequest) m;
|
||||
return "region= " + getStringForByteString(r.getRegion().getValue()) +
|
||||
|
|
|
@ -63,14 +63,12 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
|
||||
|
@ -101,6 +99,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLa
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Triple;
|
||||
import org.mortbay.log.Log;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
|
@ -131,7 +130,6 @@ public final class RequestConverter {
|
|||
GetRequest.Builder builder = GetRequest.newBuilder();
|
||||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setClosestRowBefore(true);
|
||||
builder.setRegion(region);
|
||||
|
||||
Column.Builder columnBuilder = Column.newBuilder();
|
||||
|
@ -140,62 +138,29 @@ public final class RequestConverter {
|
|||
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<Get> 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
|
||||
*
|
||||
|
@ -358,17 +323,18 @@ public final class RequestConverter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a protocol buffer MultiRequest for a row mutations
|
||||
*
|
||||
* Create a protocol buffer MultiRequest for row mutations.
|
||||
* Does not propagate Action absolute position. Does not set atomic action on the created
|
||||
* RegionAtomic. Caller should do that if wanted.
|
||||
* @param regionName
|
||||
* @param rowMutations
|
||||
* @return a multi request
|
||||
* @return a data-laden RegionMutation.Builder
|
||||
* @throws IOException
|
||||
*/
|
||||
public static MultiRequest buildMultiRequest(final byte[] regionName,
|
||||
public static RegionAction.Builder buildRegionAction(final byte [] regionName,
|
||||
final RowMutations rowMutations)
|
||||
throws IOException {
|
||||
MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, true);
|
||||
RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName);
|
||||
for (Mutation mutation: rowMutations.getMutations()) {
|
||||
MutationType mutateType = null;
|
||||
if (mutation instanceof Put) {
|
||||
|
@ -380,25 +346,26 @@ public final class RequestConverter {
|
|||
mutation.getClass().getName());
|
||||
}
|
||||
MutationProto mp = ProtobufUtil.toMutation(mutateType, mutation);
|
||||
builder.addAction(MultiAction.newBuilder().setMutation(mp).build());
|
||||
builder.addAction(ClientProtos.Action.newBuilder().setMutation(mp).build());
|
||||
}
|
||||
return builder.build();
|
||||
return builder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a protocol buffer MultiRequest for row mutations that does not hold data. Data/Cells
|
||||
* are carried outside of protobuf. Return references to the Cells in <code>cells</code> param
|
||||
*
|
||||
* are carried outside of protobuf. Return references to the Cells in <code>cells</code> 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<CellScannable> 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 @@ public final class RequestConverter {
|
|||
}
|
||||
MutationProto mp = ProtobufUtil.toMutationNoData(type, mutation);
|
||||
cells.add(mutation);
|
||||
builder.addAction(MultiAction.newBuilder().setMutation(mp).build());
|
||||
builder.addAction(ClientProtos.Action.newBuilder().setMutation(mp).build());
|
||||
}
|
||||
return builder.build();
|
||||
return builder;
|
||||
}
|
||||
|
||||
private static MultiRequest.Builder getMultiRequestBuilderWithRegionAndAtomicSet(final byte [] regionName,
|
||||
final boolean atomic) {
|
||||
MultiRequest.Builder builder = MultiRequest.newBuilder();
|
||||
private static RegionAction.Builder getRegionActionBuilderWithRegion(final byte [] regionName) {
|
||||
RegionAction.Builder builder = RegionAction.newBuilder();
|
||||
RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
return builder.setAtomic(atomic);
|
||||
return builder;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -510,39 +476,43 @@ public final class RequestConverter {
|
|||
|
||||
/**
|
||||
* Create a protocol buffer multi request for a list of actions.
|
||||
* RowMutations in the list (if any) will be ignored.
|
||||
* Propagates Actions original index.
|
||||
*
|
||||
* @param regionName
|
||||
* @param actions
|
||||
* @return a multi request
|
||||
* @throws IOException
|
||||
*/
|
||||
public static <R> MultiRequest buildMultiRequest(final byte[] regionName,
|
||||
public static <R> RegionAction.Builder buildRegionAction(final byte[] regionName,
|
||||
final List<Action<R>> actions)
|
||||
throws IOException {
|
||||
MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false);
|
||||
RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName);
|
||||
for (Action<R> 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 @@ public final class RequestConverter {
|
|||
* carried by protobuf. We return references to the data by adding them to the passed in
|
||||
* <code>data</code> param.
|
||||
*
|
||||
* RowMutations in the list (if any) will be ignored.
|
||||
* <p>Propagates Actions original index.
|
||||
*
|
||||
* @param regionName
|
||||
* @param actions
|
||||
|
@ -561,20 +531,22 @@ public final class RequestConverter {
|
|||
* @return a multi request that does not carry any data.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static <R> MultiRequest buildNoDataMultiRequest(final byte[] regionName,
|
||||
public static <R> RegionAction.Builder buildNoDataRegionAction(final byte[] regionName,
|
||||
final List<Action<R>> actions, final List<CellScannable> cells)
|
||||
throws IOException {
|
||||
MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false);
|
||||
RegionAction.Builder builder = getRegionActionBuilderWithRegion(regionName);
|
||||
for (Action<R> 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 @@ public final class RequestConverter {
|
|||
// metadata only in the pb and then send the kv along the side in cells.
|
||||
if (size > 0) {
|
||||
cells.add(d);
|
||||
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d));
|
||||
builder.addAction(actionBuilder.
|
||||
setMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d)));
|
||||
} else {
|
||||
protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, d));
|
||||
builder.addAction(actionBuilder.
|
||||
setMutation(ProtobufUtil.toMutation(MutationType.DELETE, d)));
|
||||
}
|
||||
} else if (row instanceof Append) {
|
||||
Append a = (Append)row;
|
||||
cells.add(a);
|
||||
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a));
|
||||
builder.addAction(actionBuilder.
|
||||
setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a)));
|
||||
} else if (row instanceof Increment) {
|
||||
Increment i = (Increment)row;
|
||||
cells.add(i);
|
||||
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i));
|
||||
builder.addAction(actionBuilder.
|
||||
setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i)));
|
||||
} else if (row instanceof RowMutations) {
|
||||
continue; // ignore RowMutations
|
||||
} else {
|
||||
throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
|
||||
}
|
||||
builder.addAction(protoAction.build());
|
||||
}
|
||||
return builder.build();
|
||||
return builder;
|
||||
}
|
||||
|
||||
// End utilities for Client
|
||||
|
|
|
@ -39,14 +39,19 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.security.access.UserPermission;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
@ -68,27 +73,58 @@ public final class ResponseConverter {
|
|||
/**
|
||||
* 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 <code>proto</code>. Can be null.
|
||||
* @return the results that were in the MultiResponse (a Result or an Exception).
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<Object> 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<Object> results = new ArrayList<Object>();
|
||||
List<ActionResult> resultList = proto.getResultList();
|
||||
for (int i = 0, n = resultList.size(); i < n; i++) {
|
||||
ActionResult result = resultList.get(i);
|
||||
if (result.hasException()) {
|
||||
results.add(ProtobufUtil.toException(result.getException()));
|
||||
} else if (result.hasValue()) {
|
||||
ClientProtos.Result value = result.getValue();
|
||||
results.add(ProtobufUtil.toResult(value, cells));
|
||||
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<Integer, Object>(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<Integer, Object>(roe.getIndex(),
|
||||
ProtobufUtil.toException(roe.getException())));
|
||||
} else if (roe.hasResult()) {
|
||||
results.add(regionName, new Pair<Integer, Object>(roe.getIndex(),
|
||||
ProtobufUtil.toResult(roe.getResult(), cells)));
|
||||
} else {
|
||||
results.add(new Result());
|
||||
// no result & no exception. Unexpected.
|
||||
throw new IllegalStateException("No result & no exception roe=" + roe +
|
||||
" for region " + actions.getRegion());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
|
@ -96,16 +132,36 @@ public final class ResponseConverter {
|
|||
* Wrap a throwable to an action result.
|
||||
*
|
||||
* @param t
|
||||
* @return an action result
|
||||
* @return an action result builder
|
||||
*/
|
||||
public static 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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -172,7 +172,7 @@ public class TestAsyncProcess {
|
|||
*/
|
||||
static class MyConnectionImpl2 extends MyConnectionImpl {
|
||||
List<HRegionLocation> hrl;
|
||||
boolean usedRegions[];
|
||||
final boolean usedRegions[];
|
||||
|
||||
protected MyConnectionImpl2(List<HRegionLocation> hrl) {
|
||||
super(c);
|
||||
|
@ -475,9 +475,9 @@ public class TestAsyncProcess {
|
|||
|
||||
|
||||
private class MyCB implements AsyncProcess.AsyncProcessCallback<Object> {
|
||||
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 @@ public class TestAsyncProcess {
|
|||
*/
|
||||
@Test
|
||||
public void testThreadCreation() throws Exception {
|
||||
final int NB_REGS = 10000;
|
||||
final int NB_REGS = 100;
|
||||
List<HRegionLocation> hrls = new ArrayList<HRegionLocation>(NB_REGS);
|
||||
List<Get> gets = new ArrayList<Get>(NB_REGS);
|
||||
for (int i = 0; i < NB_REGS; i++) {
|
||||
|
@ -721,11 +721,13 @@ public class TestAsyncProcess {
|
|||
HTable ht = new HTable();
|
||||
MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
|
||||
ht.connection = con;
|
||||
|
||||
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; i<NB_REGS; i++){
|
||||
|
|
|
@ -244,11 +244,13 @@ public final class CellUtil {
|
|||
|
||||
@Override
|
||||
public Cell current() {
|
||||
if (cells == null) return null;
|
||||
return (index < 0)? null: this.cells[index];
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advance() {
|
||||
if (cells == null) return false;
|
||||
return ++index < this.cells.length;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -25,7 +25,6 @@ terminal and hit return -- the protoc compiler runs fast):
|
|||
do
|
||||
protoc -I$PROTO_DIR --java_out=$JAVA_DIR $PROTO_FILE
|
||||
done
|
||||
ll $JAVA_DIR/org/apache/hadoop/hbase/protobuf/generated
|
||||
|
||||
After you've done the above, check it in and then check it in (or post a patch
|
||||
on a JIRA with your definition file changes and the generated files).
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -3662,6 +3662,26 @@ public final class RPCProtos {
|
|||
* </pre>
|
||||
*/
|
||||
org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMetaOrBuilder getCellBlockMetaOrBuilder();
|
||||
|
||||
// optional uint32 priority = 6;
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
boolean hasPriority();
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
int getPriority();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code RequestHeader}
|
||||
|
@ -3759,6 +3779,11 @@ public final class RPCProtos {
|
|||
bitField0_ |= 0x00000010;
|
||||
break;
|
||||
}
|
||||
case 48: {
|
||||
bitField0_ |= 0x00000020;
|
||||
priority_ = input.readUInt32();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -3946,12 +3971,39 @@ public final class RPCProtos {
|
|||
return cellBlockMeta_;
|
||||
}
|
||||
|
||||
// optional uint32 priority = 6;
|
||||
public static final int PRIORITY_FIELD_NUMBER = 6;
|
||||
private int priority_;
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasPriority() {
|
||||
return ((bitField0_ & 0x00000020) == 0x00000020);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
public int getPriority() {
|
||||
return priority_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
callId_ = 0;
|
||||
traceInfo_ = org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo.getDefaultInstance();
|
||||
methodName_ = "";
|
||||
requestParam_ = false;
|
||||
cellBlockMeta_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta.getDefaultInstance();
|
||||
priority_ = 0;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -3980,6 +4032,9 @@ public final class RPCProtos {
|
|||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
output.writeMessage(5, cellBlockMeta_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000020) == 0x00000020)) {
|
||||
output.writeUInt32(6, priority_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -4009,6 +4064,10 @@ public final class RPCProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(5, cellBlockMeta_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000020) == 0x00000020)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt32Size(6, priority_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -4057,6 +4116,11 @@ public final class RPCProtos {
|
|||
result = result && getCellBlockMeta()
|
||||
.equals(other.getCellBlockMeta());
|
||||
}
|
||||
result = result && (hasPriority() == other.hasPriority());
|
||||
if (hasPriority()) {
|
||||
result = result && (getPriority()
|
||||
== other.getPriority());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -4090,6 +4154,10 @@ public final class RPCProtos {
|
|||
hash = (37 * hash) + CELL_BLOCK_META_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getCellBlockMeta().hashCode();
|
||||
}
|
||||
if (hasPriority()) {
|
||||
hash = (37 * hash) + PRIORITY_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getPriority();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -4223,6 +4291,8 @@ public final class RPCProtos {
|
|||
cellBlockMetaBuilder_.clear();
|
||||
}
|
||||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
priority_ = 0;
|
||||
bitField0_ = (bitField0_ & ~0x00000020);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -4279,6 +4349,10 @@ public final class RPCProtos {
|
|||
} else {
|
||||
result.cellBlockMeta_ = cellBlockMetaBuilder_.build();
|
||||
}
|
||||
if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
|
||||
to_bitField0_ |= 0x00000020;
|
||||
}
|
||||
result.priority_ = priority_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -4312,6 +4386,9 @@ public final class RPCProtos {
|
|||
if (other.hasCellBlockMeta()) {
|
||||
mergeCellBlockMeta(other.getCellBlockMeta());
|
||||
}
|
||||
if (other.hasPriority()) {
|
||||
setPriority(other.getPriority());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -4781,6 +4858,59 @@ public final class RPCProtos {
|
|||
return cellBlockMetaBuilder_;
|
||||
}
|
||||
|
||||
// optional uint32 priority = 6;
|
||||
private int priority_ ;
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasPriority() {
|
||||
return ((bitField0_ & 0x00000020) == 0x00000020);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
public int getPriority() {
|
||||
return priority_;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
public Builder setPriority(int value) {
|
||||
bitField0_ |= 0x00000020;
|
||||
priority_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint32 priority = 6;</code>
|
||||
*
|
||||
* <pre>
|
||||
* 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
* See HConstants.
|
||||
* </pre>
|
||||
*/
|
||||
public Builder clearPriority() {
|
||||
bitField0_ = (bitField0_ & ~0x00000020);
|
||||
priority_ = 0;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:RequestHeader)
|
||||
}
|
||||
|
||||
|
@ -5797,15 +5927,15 @@ public final class RPCProtos {
|
|||
"\001(\r\"|\n\021ExceptionResponse\022\034\n\024exception_cl" +
|
||||
"ass_name\030\001 \001(\t\022\023\n\013stack_trace\030\002 \001(\t\022\020\n\010h" +
|
||||
"ostname\030\003 \001(\t\022\014\n\004port\030\004 \001(\005\022\024\n\014do_not_re",
|
||||
"try\030\005 \001(\010\"\224\001\n\rRequestHeader\022\017\n\007call_id\030\001" +
|
||||
"try\030\005 \001(\010\"\246\001\n\rRequestHeader\022\017\n\007call_id\030\001" +
|
||||
" \001(\r\022\035\n\ntrace_info\030\002 \001(\0132\t.RPCTInfo\022\023\n\013m" +
|
||||
"ethod_name\030\003 \001(\t\022\025\n\rrequest_param\030\004 \001(\010\022" +
|
||||
"\'\n\017cell_block_meta\030\005 \001(\0132\016.CellBlockMeta" +
|
||||
"\"q\n\016ResponseHeader\022\017\n\007call_id\030\001 \001(\r\022%\n\te" +
|
||||
"xception\030\002 \001(\0132\022.ExceptionResponse\022\'\n\017ce" +
|
||||
"ll_block_meta\030\003 \001(\0132\016.CellBlockMetaB<\n*o" +
|
||||
"rg.apache.hadoop.hbase.protobuf.generate" +
|
||||
"dB\tRPCProtosH\001\240\001\001"
|
||||
"\022\020\n\010priority\030\006 \001(\r\"q\n\016ResponseHeader\022\017\n\007" +
|
||||
"call_id\030\001 \001(\r\022%\n\texception\030\002 \001(\0132\022.Excep" +
|
||||
"tionResponse\022\'\n\017cell_block_meta\030\003 \001(\0132\016." +
|
||||
"CellBlockMetaB<\n*org.apache.hadoop.hbase" +
|
||||
".protobuf.generatedB\tRPCProtosH\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -5841,7 +5971,7 @@ public final class RPCProtos {
|
|||
internal_static_RequestHeader_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_RequestHeader_descriptor,
|
||||
new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", });
|
||||
new java.lang.String[] { "CallId", "TraceInfo", "MethodName", "RequestParam", "CellBlockMeta", "Priority", });
|
||||
internal_static_ResponseHeader_descriptor =
|
||||
getDescriptor().getMessageTypes().get(5);
|
||||
internal_static_ResponseHeader_fieldAccessorTable = new
|
||||
|
|
|
@ -38,7 +38,10 @@ message Column {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ message Get {
|
|||
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,56 +74,22 @@ message Result {
|
|||
// 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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -318,43 +295,60 @@ message CoprocessorServiceResponse {
|
|||
required NameBytesPair value = 2;
|
||||
}
|
||||
|
||||
/**
|
||||
* An action that is part of MultiRequest.
|
||||
* This is a union type - exactly one of the fields will be set.
|
||||
*/
|
||||
message MultiAction {
|
||||
optional MutationProto mutation = 1;
|
||||
optional Get get = 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 individual action result. The result will in the
|
||||
* same order as the action in the request. If an action
|
||||
* returns a value, it is set in value field. If it doesn't
|
||||
* return anything, the result will be empty. If an action
|
||||
* fails to execute due to any exception, the exception
|
||||
* is returned as a stringified parameter.
|
||||
* Actions to run against a Region.
|
||||
*/
|
||||
message ActionResult {
|
||||
optional Result value = 1;
|
||||
message RegionAction {
|
||||
required RegionSpecifier region = 1;
|
||||
// When set, run mutations as atomic unit.
|
||||
optional bool atomic = 2;
|
||||
repeated Action action = 3;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 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;
|
||||
}
|
||||
|
||||
|
||||
|
@ -362,9 +356,6 @@ service ClientService {
|
|||
rpc Get(GetRequest)
|
||||
returns(GetResponse);
|
||||
|
||||
rpc MultiGet(MultiGetRequest)
|
||||
returns(MultiGetResponse);
|
||||
|
||||
rpc Mutate(MutateRequest)
|
||||
returns(MutateResponse);
|
||||
|
||||
|
|
|
@ -119,7 +119,9 @@ message RequestHeader {
|
|||
optional bool request_param = 4;
|
||||
// If present, then an encoded data block follows.
|
||||
optional CellBlockMeta cell_block_meta = 5;
|
||||
// TODO: Have client specify priority
|
||||
// 0 is NORMAL priority. 100 is HIGH. If no priority, treat it as NORMAL.
|
||||
// See HConstants.
|
||||
optional uint32 priority = 6;
|
||||
}
|
||||
|
||||
message ResponseHeader {
|
||||
|
|
|
@ -83,8 +83,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
|
|||
CompactRegionRequest.class,
|
||||
GetRequest.class,
|
||||
MutateRequest.class,
|
||||
ScanRequest.class,
|
||||
MultiRequest.class
|
||||
ScanRequest.class
|
||||
};
|
||||
|
||||
// Some caches for helping performance
|
||||
|
@ -101,7 +100,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
|
|||
if (p != null) {
|
||||
// Since we protobuf'd, and then subsequently, when we went with pb style, method names
|
||||
// are capitalized. This meant that this brittle compare of method names gotten by
|
||||
// reflection no longer matched the method names comeing in over pb. TODO: Get rid of this
|
||||
// reflection no longer matched the method names coming in over pb. TODO: Get rid of this
|
||||
// check. For now, workaround is to capitalize the names we got from reflection so they
|
||||
// have chance of matching the pb ones.
|
||||
String capitalizedMethodName = capitalize(m.getName());
|
||||
|
@ -109,7 +108,6 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
|
|||
}
|
||||
}
|
||||
this.annotatedQos = qosMap;
|
||||
|
||||
if (methodMap.get("getRegion") == null) {
|
||||
methodMap.put("hasRegion", new HashMap<Class<? extends Message>, Method>());
|
||||
methodMap.put("getRegion", new HashMap<Class<? extends Message>, Method>());
|
||||
|
@ -148,10 +146,14 @@ class AnnotationReadingPriorityFunction implements PriorityFunction {
|
|||
if (priorityByAnnotation != null) {
|
||||
return priorityByAnnotation;
|
||||
}
|
||||
|
||||
if (param == null) {
|
||||
return HConstants.NORMAL_QOS;
|
||||
}
|
||||
if (methodName.equalsIgnoreCase("multi") && param instanceof MultiRequest) {
|
||||
// The multi call has its priority set in the header. All calls should work this way but
|
||||
// only this one has been converted so far. No priority == NORMAL_QOS.
|
||||
return header.hasPriority()? header.getPriority(): HConstants.NORMAL_QOS;
|
||||
}
|
||||
String cls = param.getClass().getName();
|
||||
Class<? extends Message> rpcArgClass = argumentToClassMap.get(cls);
|
||||
RegionSpecifier regionSpecifier = null;
|
||||
|
|
|
@ -73,16 +73,16 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionTooBusyException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionTooBusyException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -113,11 +113,9 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
|
|||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
|
@ -4331,7 +4329,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
List<Cell> results = get(get, true);
|
||||
return Result.create(results);
|
||||
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null);
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.Chore;
|
|||
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
|
@ -149,7 +150,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodes
|
|||
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;
|
||||
|
@ -158,14 +158,15 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServic
|
|||
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;
|
||||
|
@ -2757,7 +2758,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
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 "
|
||||
|
@ -2768,13 +2769,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
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);
|
||||
}
|
||||
|
@ -2782,9 +2783,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
}
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
@ -2796,62 +2798,6 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
|
@ -3300,106 +3246,134 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
// It is also the conduit via which we pass back data.
|
||||
PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
|
||||
CellScanner cellScanner = controller != null ? controller.cellScanner(): null;
|
||||
// Clear scanner so we are not holding on to reference across call.
|
||||
if (controller != null) controller.setCellScanner(null);
|
||||
|
||||
// this will contain all the cells that we need to return. It's created later, if needed.
|
||||
List<CellScannable> cellsToReturn = null;
|
||||
MultiResponse.Builder responseBuilder = MultiResponse.newBuilder();
|
||||
|
||||
for (RegionAction regionAction : request.getRegionActionList()) {
|
||||
this.requestCount.add(regionAction.getActionCount());
|
||||
RegionActionResult.Builder regionActionResultBuilder = RegionActionResult.newBuilder();
|
||||
HRegion region;
|
||||
try {
|
||||
HRegion region = getRegion(request.getRegion());
|
||||
MultiResponse.Builder builder = MultiResponse.newBuilder();
|
||||
List<MutationProto> mutations = new ArrayList<MutationProto>(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);
|
||||
region = getRegion(regionAction.getRegion());
|
||||
} catch (IOException e) {
|
||||
regionActionResultBuilder.setException(ResponseConverter.buildException(e));
|
||||
responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
|
||||
continue; // For this region it's a failure.
|
||||
}
|
||||
}
|
||||
// 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<CellScannable>(request.getActionCount());
|
||||
for (ClientProtos.MultiAction actionUnion : request.getActionList()) {
|
||||
this.requestCount.increment();
|
||||
ClientProtos.Result result = null;
|
||||
|
||||
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 {
|
||||
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);
|
||||
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 {
|
||||
LOG.warn("Error: invalid action: " + actionUnion + ". "
|
||||
+ "it must be a Get, Mutate, or Exec.");
|
||||
throw new DoNotRetryIOException("Invalid action, "
|
||||
+ "it must be a Get, Mutate, or Exec.");
|
||||
}
|
||||
if (result != null) {
|
||||
if (resultBuilder == null) {
|
||||
resultBuilder = ActionResult.newBuilder();
|
||||
} else {
|
||||
resultBuilder.clear();
|
||||
}
|
||||
resultBuilder.setValue(result);
|
||||
builder.addResult(resultBuilder.build());
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
builder.addResult(ResponseConverter.buildActionResult(ie));
|
||||
}
|
||||
}
|
||||
if (!mutations.isEmpty()) {
|
||||
doBatchOp(builder, region, mutations, cellScanner);
|
||||
// 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 builder.build();
|
||||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
return responseBuilder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Run through the regionMutation <code>rm</code> 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 <code>cellScanner</code> passed
|
||||
*/
|
||||
private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
|
||||
final RegionAction actions, final CellScanner cellScanner,
|
||||
final RegionActionResult.Builder builder, List<CellScannable> 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<ClientProtos.Action> 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<ClientProtos.Action>(actions.getActionCount());
|
||||
}
|
||||
mutations.add(action);
|
||||
break;
|
||||
default:
|
||||
throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
|
||||
}
|
||||
} else {
|
||||
throw new HBaseIOException("Unexpected Action type");
|
||||
}
|
||||
if (r != null) {
|
||||
ClientProtos.Result pbResult = null;
|
||||
if (isClientCellBlockSupport()) {
|
||||
pbResult = ProtobufUtil.toResultNoData(r);
|
||||
// Hard to guess the size here. Just make a rough guess.
|
||||
if (cellsToReturn == null) cellsToReturn = new ArrayList<CellScannable>();
|
||||
cellsToReturn.add(r);
|
||||
} else {
|
||||
pbResult = ProtobufUtil.toResult(r);
|
||||
}
|
||||
resultOrExceptionBuilder =
|
||||
ClientProtos.ResultOrException.newBuilder().setResult(pbResult);
|
||||
}
|
||||
// Could get to here and there was no result and no exception. Presumes we added
|
||||
// a Put or Delete to the collecting Mutations List for adding later. In this
|
||||
// case the corresponding ResultOrException instance for the Put or Delete will be added
|
||||
// down in the doBatchOp method call rather than up here.
|
||||
} catch (IOException ie) {
|
||||
resultOrExceptionBuilder = ResultOrException.newBuilder().
|
||||
setException(ResponseConverter.buildException(ie));
|
||||
}
|
||||
if (resultOrExceptionBuilder != null) {
|
||||
// Propagate index.
|
||||
resultOrExceptionBuilder.setIndex(action.getIndex());
|
||||
builder.addResultOrException(resultOrExceptionBuilder.build());
|
||||
}
|
||||
}
|
||||
// Finish up any outstanding mutations
|
||||
if (mutations != null && !mutations.isEmpty()) {
|
||||
doBatchOp(builder, region, mutations, cellScanner);
|
||||
}
|
||||
return cellsToReturn;
|
||||
}
|
||||
|
||||
// End Client methods
|
||||
|
@ -3877,11 +3851,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
try {
|
||||
checkOpen();
|
||||
List<WALEntry> 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();
|
||||
|
@ -4062,17 +4035,17 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* @param region
|
||||
* @param mutations
|
||||
*/
|
||||
protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region,
|
||||
final List<MutationProto> mutations, final CellScanner cells) {
|
||||
protected void doBatchOp(final RegionActionResult.Builder builder, final HRegion region,
|
||||
final List<ClientProtos.Action> 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);
|
||||
|
@ -4082,7 +4055,6 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
batchContainsDelete = true;
|
||||
}
|
||||
mArray[i++] = mutation;
|
||||
builder.addResult(result);
|
||||
}
|
||||
|
||||
requestCount.add(mutations.size());
|
||||
|
@ -4092,33 +4064,33 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
|
||||
OperationStatus codes[] = region.batchMutate(mArray, false);
|
||||
for (i = 0; i < codes.length; i++) {
|
||||
int index = mutations.get(i).getIndex();
|
||||
Exception e = null;
|
||||
switch (codes[i].getOperationStatusCode()) {
|
||||
case BAD_FAMILY:
|
||||
result = ResponseConverter.buildActionResult(
|
||||
new NoSuchColumnFamilyException(codes[i].getExceptionMsg()));
|
||||
builder.setResult(i, result);
|
||||
e = new NoSuchColumnFamilyException(codes[i].getExceptionMsg());
|
||||
builder.addResultOrException(getResultOrException(e, index));
|
||||
break;
|
||||
|
||||
case SANITY_CHECK_FAILURE:
|
||||
result = ResponseConverter.buildActionResult(
|
||||
new FailedSanityCheckException(codes[i].getExceptionMsg()));
|
||||
builder.setResult(i, result);
|
||||
e = new FailedSanityCheckException(codes[i].getExceptionMsg());
|
||||
builder.addResultOrException(getResultOrException(e, index));
|
||||
break;
|
||||
|
||||
default:
|
||||
result = ResponseConverter.buildActionResult(
|
||||
new DoNotRetryIOException(codes[i].getExceptionMsg()));
|
||||
builder.setResult(i, result);
|
||||
e = new DoNotRetryIOException(codes[i].getExceptionMsg());
|
||||
builder.addResultOrException(getResultOrException(e, index));
|
||||
break;
|
||||
|
||||
case SUCCESS:
|
||||
builder.addResultOrException(getResultOrException(ClientProtos.Result.getDefaultInstance(), index));
|
||||
break;
|
||||
}
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
ActionResult result = ResponseConverter.buildActionResult(ie);
|
||||
ResultOrException resultOrException = ResponseConverter.buildActionResult(ie).build();
|
||||
for (int i = 0; i < mutations.size(); i++) {
|
||||
builder.setResult(i, result);
|
||||
builder.addResultOrException(resultOrException);
|
||||
}
|
||||
}
|
||||
long after = EnvironmentEdgeManager.currentTimeMillis();
|
||||
|
@ -4129,6 +4101,18 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
metricsRegionServer.updateDelete(after - before);
|
||||
}
|
||||
}
|
||||
private static ResultOrException getResultOrException(final ClientProtos.Result r,
|
||||
final int index) {
|
||||
return getResultOrException(ResponseConverter.buildActionResult(r), index);
|
||||
}
|
||||
private static ResultOrException getResultOrException(final Exception e, final int index) {
|
||||
return getResultOrException(ResponseConverter.buildActionResult(e), index);
|
||||
}
|
||||
|
||||
private static ResultOrException getResultOrException(final ResultOrException.Builder builder,
|
||||
final int index) {
|
||||
return builder.setIndex(index).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute a list of Put/Delete mutations. The function returns OperationStatus instead of
|
||||
|
@ -4140,8 +4124,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* exceptionMessage if any
|
||||
* @throws IOException
|
||||
*/
|
||||
protected OperationStatus[] doBatchOp(final HRegion region,
|
||||
final List<Pair<MutationType, Mutation>> mutations, boolean isReplay) throws IOException {
|
||||
protected OperationStatus [] doBatchOp(final HRegion region,
|
||||
final List<Pair<MutationType, Mutation>> mutations, boolean isReplay)
|
||||
throws IOException {
|
||||
Mutation[] mArray = new Mutation[mutations.size()];
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||
|
@ -4175,32 +4160,35 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* 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<MutationProto> mutations,
|
||||
protected void mutateRows(final HRegion region, final List<ClientProtos.Action> 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);
|
||||
|
@ -4402,7 +4390,6 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
/**
|
||||
* Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
|
||||
* @param encodedRegionName
|
||||
* @throws IOException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
|
||||
|
|
|
@ -23,8 +23,6 @@ import java.util.ArrayList;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -32,35 +30,23 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.client.Action;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.RegionServerCallable;
|
||||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCaller;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -4356,8 +4356,7 @@ public class TestFromClientSide {
|
|||
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]));
|
||||
|
|
|
@ -43,7 +43,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -51,12 +50,12 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.master.ClusterStatusPublisher;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
|
@ -447,12 +446,12 @@ public class TestHCM {
|
|||
try {
|
||||
table.put(put3);
|
||||
Assert.fail("Unreachable point");
|
||||
}catch (RetriesExhaustedWithDetailsException e){
|
||||
} catch (RetriesExhaustedWithDetailsException e){
|
||||
LOG.info("Put done, exception caught: " + e.getClass());
|
||||
Assert.assertEquals(1, e.getNumExceptions());
|
||||
Assert.assertArrayEquals(e.getRow(0).getRow(), ROW);
|
||||
}
|
||||
Assert.assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
|
||||
Assert.assertNotNull("Cached connection is null", conn.getCachedLocation(TABLE_NAME, ROW));
|
||||
Assert.assertEquals(
|
||||
"Previous server was "+curServer.getServerName().getHostAndPort(),
|
||||
destServerName.getPort(), conn.getCachedLocation(TABLE_NAME, ROW).getPort());
|
||||
|
|
|
@ -179,7 +179,6 @@ public class TestMultiParallel {
|
|||
for (Row get : gets) {
|
||||
singleRes.add(table.get((Get) get));
|
||||
}
|
||||
|
||||
// Compare results
|
||||
Assert.assertEquals(singleRes.size(), multiRes.length);
|
||||
for (int i = 0; i < singleRes.size(); i++) {
|
||||
|
@ -332,16 +331,20 @@ public class TestMultiParallel {
|
|||
validateSizeAndEmpty(results, KEYS.length);
|
||||
|
||||
if (true) {
|
||||
int liveRScount = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()
|
||||
.size();
|
||||
int liveRScount = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size();
|
||||
assert liveRScount > 0;
|
||||
JVMClusterUtil.RegionServerThread liveRS = UTIL.getMiniHBaseCluster()
|
||||
.getLiveRegionServerThreads().get(0);
|
||||
liveRS.getRegionServer().abort("Aborting for tests",
|
||||
new Exception("testBatchWithPut"));
|
||||
|
||||
JVMClusterUtil.RegionServerThread liveRS =
|
||||
UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().get(0);
|
||||
liveRS.getRegionServer().abort("Aborting for tests", new Exception("testBatchWithPut"));
|
||||
puts = constructPutRequests();
|
||||
try {
|
||||
results = table.batch(puts);
|
||||
} catch (RetriesExhaustedWithDetailsException ree) {
|
||||
LOG.info(ree.getExhaustiveDescription());
|
||||
throw ree;
|
||||
} finally {
|
||||
table.close();
|
||||
}
|
||||
validateSizeAndEmpty(results, KEYS.length);
|
||||
}
|
||||
|
||||
|
@ -597,6 +600,4 @@ public class TestMultiParallel {
|
|||
validateEmpty(result);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -21,12 +21,9 @@ package org.apache.hadoop.hbase.ipc;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyInt;
|
||||
import static org.mockito.Matchers.anyObject;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.internal.verification.VerificationModeFactory.times;
|
||||
|
@ -36,12 +33,9 @@ import java.net.InetSocketAddress;
|
|||
import java.net.Socket;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import javax.net.SocketFactory;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -64,6 +58,7 @@ import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponsePr
|
|||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -72,11 +67,12 @@ import org.apache.hadoop.net.NetUtils;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Matchers;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.BlockingService;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
|
@ -321,7 +317,7 @@ public class TestIPC {
|
|||
for (int i = 0; i < cycles; i++) {
|
||||
List<CellScannable> cells = new ArrayList<CellScannable>();
|
||||
// Message param = RequestConverter.buildMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm);
|
||||
Message param = RequestConverter.buildNoDataMultiRequest(
|
||||
ClientProtos.RegionAction.Builder builder = RequestConverter.buildNoDataRegionAction(
|
||||
HConstants.EMPTY_BYTE_ARRAY, rm, cells);
|
||||
CellScanner cellScanner = CellUtil.createCellScanner(cells);
|
||||
if (i % 1000 == 0) {
|
||||
|
@ -331,7 +327,7 @@ public class TestIPC {
|
|||
// "Thread dump " + Thread.currentThread().getName());
|
||||
}
|
||||
Pair<Message, CellScanner> 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()) {
|
||||
|
|
|
@ -79,10 +79,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ
|
|||
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;
|
||||
|
@ -346,20 +343,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiGetResponse multiGet(RpcController controller, MultiGetRequest requests)
|
||||
throws ServiceException {
|
||||
byte[] regionName = requests.getRegion().getValue().toByteArray();
|
||||
Map<byte [], Result> 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
|
||||
|
|
|
@ -289,6 +289,9 @@ public class TestSnapshotFromMaster {
|
|||
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 @@ public class TestSnapshotFromMaster {
|
|||
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);
|
||||
|
||||
|
|
|
@ -48,7 +48,7 @@ public class TestHRegionBusyWait extends TestHRegion {
|
|||
/**
|
||||
* 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);
|
||||
|
|
|
@ -20,8 +20,8 @@ import static org.junit.Assert.assertEquals;
|
|||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -43,11 +43,19 @@ public class TestQosFunction {
|
|||
checkMethod("ReplicateWALEntry", HConstants.REPLICATION_QOS, qosFunction);
|
||||
// Set method name in pb style with the method name capitalized.
|
||||
checkMethod("OpenRegion", HConstants.HIGH_QOS, qosFunction);
|
||||
// Check multi works.
|
||||
checkMethod("Multi", HConstants.NORMAL_QOS, qosFunction, MultiRequest.getDefaultInstance());
|
||||
}
|
||||
|
||||
private void checkMethod(final String methodName, final int expected, final AnnotationReadingPriorityFunction qosf) {
|
||||
private void checkMethod(final String methodName, final int expected,
|
||||
final AnnotationReadingPriorityFunction qosf) {
|
||||
checkMethod(methodName, expected, qosf, null);
|
||||
}
|
||||
|
||||
private void checkMethod(final String methodName, final int expected,
|
||||
final AnnotationReadingPriorityFunction qosf, final Message param) {
|
||||
RequestHeader.Builder builder = RequestHeader.newBuilder();
|
||||
builder.setMethodName(methodName);
|
||||
assertEquals(methodName, expected, qosf.getPriority(builder.build(), null));
|
||||
assertEquals(methodName, expected, qosf.getPriority(builder.build(), param));
|
||||
}
|
||||
}
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.rest;
|
|||
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.BeforeClass;
|
|||
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();
|
||||
|
|
Loading…
Reference in New Issue