HBASE-10277 refactor AsyncProcess
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1564832 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1c46c16cc0
commit
7898e68fd4
File diff suppressed because it is too large
Load Diff
|
@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||||
|
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
||||||
|
@ -524,4 +525,9 @@ public interface HConnection extends Abortable, Closeable {
|
||||||
* @return Nonce generator for this HConnection; may be null if disabled in configuration.
|
* @return Nonce generator for this HConnection; may be null if disabled in configuration.
|
||||||
*/
|
*/
|
||||||
public NonceGenerator getNonceGenerator();
|
public NonceGenerator getNonceGenerator();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Default AsyncProcess associated with this connection.
|
||||||
|
*/
|
||||||
|
AsyncProcess getAsyncProcess();
|
||||||
}
|
}
|
|
@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.TableNotEnabledException;
|
import org.apache.hadoop.hbase.TableNotEnabledException;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
|
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
||||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||||
|
@ -76,42 +77,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
|
||||||
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
|
||||||
import org.apache.hadoop.hbase.security.User;
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
@ -233,6 +198,18 @@ public class HConnectionManager {
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Dummy nonce generator for disabled nonces. */
|
||||||
|
static class NoNonceGenerator implements NonceGenerator {
|
||||||
|
@Override
|
||||||
|
public long getNonceGroup() {
|
||||||
|
return HConstants.NO_NONCE;
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public long newNonce() {
|
||||||
|
return HConstants.NO_NONCE;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Non-instantiable.
|
* Non-instantiable.
|
||||||
*/
|
*/
|
||||||
|
@ -574,6 +551,7 @@ public class HConnectionManager {
|
||||||
final int rpcTimeout;
|
final int rpcTimeout;
|
||||||
private NonceGenerator nonceGenerator = null;
|
private NonceGenerator nonceGenerator = null;
|
||||||
private final int prefetchRegionLimit;
|
private final int prefetchRegionLimit;
|
||||||
|
private final AsyncProcess asyncProcess;
|
||||||
|
|
||||||
private volatile boolean closed;
|
private volatile boolean closed;
|
||||||
private volatile boolean aborted;
|
private volatile boolean aborted;
|
||||||
|
@ -687,18 +665,6 @@ public class HConnectionManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Dummy nonce generator for disabled nonces. */
|
|
||||||
private static class NoNonceGenerator implements NonceGenerator {
|
|
||||||
@Override
|
|
||||||
public long getNonceGroup() {
|
|
||||||
return HConstants.NO_NONCE;
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
public long newNonce() {
|
|
||||||
return HConstants.NO_NONCE;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* For tests.
|
* For tests.
|
||||||
*/
|
*/
|
||||||
|
@ -722,6 +688,7 @@ public class HConnectionManager {
|
||||||
} else {
|
} else {
|
||||||
this.nonceGenerator = new NoNonceGenerator();
|
this.nonceGenerator = new NoNonceGenerator();
|
||||||
}
|
}
|
||||||
|
this.asyncProcess = createAsyncProcess(this.conf);
|
||||||
|
|
||||||
this.prefetchRegionLimit = conf.getInt(
|
this.prefetchRegionLimit = conf.getInt(
|
||||||
HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
|
HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
|
||||||
|
@ -2342,17 +2309,11 @@ public class HConnectionManager {
|
||||||
Batch.Callback<R> callback)
|
Batch.Callback<R> callback)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
// To fulfill the original contract, we have a special callback. This callback
|
AsyncRequestFuture ars = this.asyncProcess.submitAll(
|
||||||
// will set the results in the Object array.
|
pool, tableName, list, callback, results);
|
||||||
ObjectResultFiller<R> cb = new ObjectResultFiller<R>(results, callback);
|
ars.waitUntilDone();
|
||||||
AsyncProcess<?> asyncProcess = createAsyncProcess(tableName, pool, cb, conf);
|
if (ars.hasError()) {
|
||||||
|
throw ars.getErrors();
|
||||||
// We're doing a submit all. This way, the originalIndex will match the initial list.
|
|
||||||
asyncProcess.submitAll(list);
|
|
||||||
asyncProcess.waitUntilDone();
|
|
||||||
|
|
||||||
if (asyncProcess.hasError()) {
|
|
||||||
throw asyncProcess.getErrors();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2368,51 +2329,17 @@ public class HConnectionManager {
|
||||||
processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
|
processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
// For tests.
|
// For tests to override.
|
||||||
protected <R> AsyncProcess createAsyncProcess(TableName tableName, ExecutorService pool,
|
protected AsyncProcess createAsyncProcess(Configuration conf) {
|
||||||
AsyncProcess.AsyncProcessCallback<R> callback, Configuration conf) {
|
// No default pool available.
|
||||||
return new AsyncProcess<R>(this, tableName, pool, callback, conf,
|
return new AsyncProcess(
|
||||||
RpcRetryingCallerFactory.instantiate(conf));
|
this, conf, this.batchPool, RpcRetryingCallerFactory.instantiate(conf), false);
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Fill the result array for the interfaces using it.
|
|
||||||
*/
|
|
||||||
private static class ObjectResultFiller<Res>
|
|
||||||
implements AsyncProcess.AsyncProcessCallback<Res> {
|
|
||||||
|
|
||||||
private final Object[] results;
|
|
||||||
private Batch.Callback<Res> callback;
|
|
||||||
|
|
||||||
ObjectResultFiller(Object[] results, Batch.Callback<Res> callback) {
|
|
||||||
this.results = results;
|
|
||||||
this.callback = callback;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void success(int pos, byte[] region, Row row, Res result) {
|
public AsyncProcess getAsyncProcess() {
|
||||||
assert pos < results.length;
|
return asyncProcess;
|
||||||
results[pos] = result;
|
|
||||||
if (callback != null) {
|
|
||||||
callback.update(region, row.getRow(), result);
|
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean failure(int pos, Row row, Throwable t) {
|
|
||||||
assert pos < results.length;
|
|
||||||
results[pos] = t;
|
|
||||||
//Batch.Callback<Res> was not called on failure in 0.94. We keep this.
|
|
||||||
return true; // we want to have this failure in the failures list.
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean retriableFailure(int originalIndex, Row row, Throwable exception) {
|
|
||||||
return true; // we retry
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Return the number of cached region for a table. It will only be called
|
* Return the number of cached region for a table. It will only be called
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
|
||||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||||
|
@ -133,7 +134,9 @@ public class HTable implements HTableInterface {
|
||||||
private final boolean cleanupConnectionOnClose; // close the connection in close()
|
private final boolean cleanupConnectionOnClose; // close the connection in close()
|
||||||
|
|
||||||
/** The Async process for puts with autoflush set to false or multiputs */
|
/** The Async process for puts with autoflush set to false or multiputs */
|
||||||
protected AsyncProcess<Object> ap;
|
protected AsyncProcess ap;
|
||||||
|
/** The Async process for batch */
|
||||||
|
protected AsyncProcess multiAp;
|
||||||
private RpcRetryingCallerFactory rpcCallerFactory;
|
private RpcRetryingCallerFactory rpcCallerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -340,8 +343,9 @@ public class HTable implements HTableInterface {
|
||||||
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
|
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
|
||||||
|
|
||||||
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(configuration);
|
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(configuration);
|
||||||
ap = new AsyncProcess<Object>(connection, tableName, pool, null,
|
// puts need to track errors globally due to how the APIs currently work.
|
||||||
configuration, rpcCallerFactory);
|
ap = new AsyncProcess(connection, configuration, pool, rpcCallerFactory, true);
|
||||||
|
multiAp = this.connection.getAsyncProcess();
|
||||||
|
|
||||||
this.maxKeyValueSize = this.configuration.getInt(
|
this.maxKeyValueSize = this.configuration.getInt(
|
||||||
"hbase.client.keyvalue.maxsize", -1);
|
"hbase.client.keyvalue.maxsize", -1);
|
||||||
|
@ -791,7 +795,11 @@ public class HTable implements HTableInterface {
|
||||||
@Override
|
@Override
|
||||||
public void batch(final List<? extends Row> actions, final Object[] results)
|
public void batch(final List<? extends Row> actions, final Object[] results)
|
||||||
throws InterruptedException, IOException {
|
throws InterruptedException, IOException {
|
||||||
batchCallback(actions, results, null);
|
AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, actions, null, results);
|
||||||
|
ars.waitUntilDone();
|
||||||
|
if (ars.hasError()) {
|
||||||
|
throw ars.getErrors();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -802,7 +810,9 @@ public class HTable implements HTableInterface {
|
||||||
@Override
|
@Override
|
||||||
public Object[] batch(final List<? extends Row> actions)
|
public Object[] batch(final List<? extends Row> actions)
|
||||||
throws InterruptedException, IOException {
|
throws InterruptedException, IOException {
|
||||||
return batchCallback(actions, null);
|
Object[] results = new Object[actions.size()];
|
||||||
|
batch(actions, results);
|
||||||
|
return results;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -911,6 +921,9 @@ public class HTable implements HTableInterface {
|
||||||
* @throws InterruptedIOException if we were interrupted.
|
* @throws InterruptedIOException if we were interrupted.
|
||||||
*/
|
*/
|
||||||
private void doPut(Put put) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
|
private void doPut(Put put) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
|
||||||
|
// This behavior is highly non-intuitive... it does not protect us against
|
||||||
|
// 94-incompatible behavior, which is a timing issue because hasError, the below code
|
||||||
|
// and setter of hasError are not synchronized. Perhaps it should be removed.
|
||||||
if (ap.hasError()) {
|
if (ap.hasError()) {
|
||||||
writeAsyncBuffer.add(put);
|
writeAsyncBuffer.add(put);
|
||||||
backgroundFlushCommits(true);
|
backgroundFlushCommits(true);
|
||||||
|
@ -938,30 +951,22 @@ public class HTable implements HTableInterface {
|
||||||
InterruptedIOException, RetriesExhaustedWithDetailsException {
|
InterruptedIOException, RetriesExhaustedWithDetailsException {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
do {
|
if (!synchronous) {
|
||||||
ap.submit(writeAsyncBuffer, true);
|
ap.submit(tableName, writeAsyncBuffer, true, null, false);
|
||||||
} while (synchronous && !writeAsyncBuffer.isEmpty());
|
|
||||||
|
|
||||||
if (synchronous) {
|
|
||||||
ap.waitUntilDone();
|
|
||||||
}
|
|
||||||
|
|
||||||
if (ap.hasError()) {
|
if (ap.hasError()) {
|
||||||
LOG.debug(tableName + ": One or more of the operations have failed -" +
|
LOG.debug(tableName + ": One or more of the operations have failed -" +
|
||||||
" waiting for all operation in progress to finish (successfully or not)");
|
" waiting for all operation in progress to finish (successfully or not)");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (synchronous || ap.hasError()) {
|
||||||
while (!writeAsyncBuffer.isEmpty()) {
|
while (!writeAsyncBuffer.isEmpty()) {
|
||||||
ap.submit(writeAsyncBuffer, true);
|
ap.submit(tableName, writeAsyncBuffer, true, null, false);
|
||||||
}
|
}
|
||||||
ap.waitUntilDone();
|
List<Row> failedRows = clearBufferOnFail ? null : writeAsyncBuffer;
|
||||||
|
RetriesExhaustedWithDetailsException error = ap.waitForAllPreviousOpsAndReset(failedRows);
|
||||||
if (!clearBufferOnFail) {
|
if (error != null) {
|
||||||
// if clearBufferOnFailed is not set, we're supposed to keep the failed operation in the
|
throw error;
|
||||||
// write buffer. This is a questionable feature kept here for backward compatibility
|
|
||||||
writeAsyncBuffer.addAll(ap.getFailedOperations());
|
|
||||||
}
|
}
|
||||||
RetriesExhaustedWithDetailsException e = ap.getErrors();
|
|
||||||
ap.clearErrors();
|
|
||||||
throw e;
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
currentWriteBufferSize = 0;
|
currentWriteBufferSize = 0;
|
||||||
|
@ -1301,8 +1306,7 @@ public class HTable implements HTableInterface {
|
||||||
*/
|
*/
|
||||||
public void processBatch(final List<? extends Row> list, final Object[] results)
|
public void processBatch(final List<? extends Row> list, final Object[] results)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
|
this.batch(list, results);
|
||||||
this.processBatchCallback(list, results, null);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
@ -35,10 +36,9 @@ import org.apache.hadoop.hbase.util.Pair;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class MultiResponse {
|
public class MultiResponse {
|
||||||
|
|
||||||
// map of regionName to list of (Results paired to the original index for that
|
// map of regionName to map of Results by the original index for that Result
|
||||||
// Result)
|
private Map<byte[], Map<Integer, Object>> results =
|
||||||
private Map<byte[], List<Pair<Integer, Object>>> results =
|
new TreeMap<byte[], Map<Integer, Object>>(Bytes.BYTES_COMPARATOR);
|
||||||
new TreeMap<byte[], List<Pair<Integer, Object>>>(Bytes.BYTES_COMPARATOR);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The server can send us a failure for the region itself, instead of individual failure.
|
* The server can send us a failure for the region itself, instead of individual failure.
|
||||||
|
@ -56,7 +56,7 @@ public class MultiResponse {
|
||||||
*/
|
*/
|
||||||
public int size() {
|
public int size() {
|
||||||
int size = 0;
|
int size = 0;
|
||||||
for (Collection<?> c : results.values()) {
|
for (Map<?,?> c : results.values()) {
|
||||||
size += c.size();
|
size += c.size();
|
||||||
}
|
}
|
||||||
return size;
|
return size;
|
||||||
|
@ -66,25 +66,19 @@ public class MultiResponse {
|
||||||
* Add the pair to the container, grouped by the regionName
|
* Add the pair to the container, grouped by the regionName
|
||||||
*
|
*
|
||||||
* @param regionName
|
* @param regionName
|
||||||
* @param r
|
* @param index the original index of the Action (request).
|
||||||
* First item in the pair is the original index of the Action
|
* @param result the result; will be empty for successful Put and Delete actions.
|
||||||
* (request). Second item is the Result. Result will be empty for
|
|
||||||
* successful Put and Delete actions.
|
|
||||||
*/
|
*/
|
||||||
public void add(byte[] regionName, Pair<Integer, Object> r) {
|
public void add(byte[] regionName, int originalIndex, Object resOrEx) {
|
||||||
List<Pair<Integer, Object>> rs = results.get(regionName);
|
Map<Integer, Object> rs = results.get(regionName);
|
||||||
if (rs == null) {
|
if (rs == null) {
|
||||||
rs = new ArrayList<Pair<Integer, Object>>();
|
rs = new HashMap<Integer, Object>();
|
||||||
results.put(regionName, rs);
|
results.put(regionName, rs);
|
||||||
}
|
}
|
||||||
rs.add(r);
|
rs.put(originalIndex, resOrEx);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void add(byte []regionName, int originalIndex, Object resOrEx) {
|
public Map<byte[], Map<Integer, Object>> getResults() {
|
||||||
add(regionName, new Pair<Integer,Object>(originalIndex, resOrEx));
|
|
||||||
}
|
|
||||||
|
|
||||||
public Map<byte[], List<Pair<Integer, Object>>> getResults() {
|
|
||||||
return results;
|
return results;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -48,11 +48,11 @@ import java.util.Set;
|
||||||
public class RetriesExhaustedWithDetailsException
|
public class RetriesExhaustedWithDetailsException
|
||||||
extends RetriesExhaustedException {
|
extends RetriesExhaustedException {
|
||||||
List<Throwable> exceptions;
|
List<Throwable> exceptions;
|
||||||
List<? extends Row> actions;
|
List<Row> actions;
|
||||||
List<String> hostnameAndPort;
|
List<String> hostnameAndPort;
|
||||||
|
|
||||||
public RetriesExhaustedWithDetailsException(List<Throwable> exceptions,
|
public RetriesExhaustedWithDetailsException(List<Throwable> exceptions,
|
||||||
List<? extends Row> actions,
|
List<Row> actions,
|
||||||
List<String> hostnameAndPort) {
|
List<String> hostnameAndPort) {
|
||||||
super("Failed " + exceptions.size() + " action" +
|
super("Failed " + exceptions.size() + " action" +
|
||||||
pluralize(exceptions) + ": " +
|
pluralize(exceptions) + ": " +
|
||||||
|
|
|
@ -117,11 +117,9 @@ public final class ResponseConverter {
|
||||||
|
|
||||||
for (ResultOrException roe : actionResult.getResultOrExceptionList()) {
|
for (ResultOrException roe : actionResult.getResultOrExceptionList()) {
|
||||||
if (roe.hasException()) {
|
if (roe.hasException()) {
|
||||||
results.add(regionName, new Pair<Integer, Object>(roe.getIndex(),
|
results.add(regionName, roe.getIndex(), ProtobufUtil.toException(roe.getException()));
|
||||||
ProtobufUtil.toException(roe.getException())));
|
|
||||||
} else if (roe.hasResult()) {
|
} else if (roe.hasResult()) {
|
||||||
results.add(regionName, new Pair<Integer, Object>(roe.getIndex(),
|
results.add(regionName, roe.getIndex(), ProtobufUtil.toResult(roe.getResult(), cells));
|
||||||
ProtobufUtil.toResult(roe.getResult(), cells)));
|
|
||||||
} else {
|
} else {
|
||||||
// no result & no exception. Unexpected.
|
// no result & no exception. Unexpected.
|
||||||
throw new IllegalStateException("No result & no exception roe=" + roe +
|
throw new IllegalStateException("No result & no exception roe=" + roe +
|
||||||
|
|
|
@ -27,6 +27,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HRegionLocation;
|
import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.MediumTests;
|
import org.apache.hadoop.hbase.MediumTests;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
|
||||||
|
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||||
|
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -73,9 +76,33 @@ public class TestAsyncProcess {
|
||||||
private static final String success = "success";
|
private static final String success = "success";
|
||||||
private static Exception failure = new Exception("failure");
|
private static Exception failure = new Exception("failure");
|
||||||
|
|
||||||
static class MyAsyncProcess<Res> extends AsyncProcess<Res> {
|
static class MyAsyncProcess extends AsyncProcess {
|
||||||
final AtomicInteger nbMultiResponse = new AtomicInteger();
|
final AtomicInteger nbMultiResponse = new AtomicInteger();
|
||||||
final AtomicInteger nbActions = new AtomicInteger();
|
final AtomicInteger nbActions = new AtomicInteger();
|
||||||
|
public List<AsyncRequestFuture> allReqs = new ArrayList<AsyncRequestFuture>();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected <Res> AsyncRequestFutureImpl<Res> createAsyncRequestFuture(TableName tableName,
|
||||||
|
List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
|
||||||
|
Batch.Callback<Res> callback, Object[] results, boolean needResults) {
|
||||||
|
// Test HTable has tableName of null, so pass DUMMY_TABLE
|
||||||
|
AsyncRequestFutureImpl<Res> r = super.createAsyncRequestFuture(
|
||||||
|
DUMMY_TABLE, actions, nonceGroup, pool, callback, results, needResults);
|
||||||
|
r.hardRetryLimit = new AtomicInteger(1);
|
||||||
|
allReqs.add(r);
|
||||||
|
return r;
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public long getRetriesRequested() {
|
||||||
|
long result = 0;
|
||||||
|
for (AsyncRequestFuture ars : allReqs) {
|
||||||
|
if (ars instanceof AsyncProcess.AsyncRequestFutureImpl) {
|
||||||
|
result += (1 - ((AsyncRequestFutureImpl<?>)ars).hardRetryLimit.get());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
static class CountingThreadFactory implements ThreadFactory {
|
static class CountingThreadFactory implements ThreadFactory {
|
||||||
final AtomicInteger nbThreads;
|
final AtomicInteger nbThreads;
|
||||||
|
@ -91,15 +118,29 @@ public class TestAsyncProcess {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public MyAsyncProcess(HConnection hc, AsyncProcessCallback<Res> callback, Configuration conf) {
|
public MyAsyncProcess(HConnection hc, Configuration conf) {
|
||||||
this(hc, callback, conf, new AtomicInteger());
|
this(hc, conf, new AtomicInteger());
|
||||||
}
|
}
|
||||||
|
|
||||||
public MyAsyncProcess(HConnection hc, AsyncProcessCallback<Res> callback, Configuration conf,
|
public MyAsyncProcess(HConnection hc, Configuration conf, AtomicInteger nbThreads) {
|
||||||
AtomicInteger nbThreads) {
|
super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
|
||||||
super(hc, DUMMY_TABLE, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
|
|
||||||
new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
|
new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
|
||||||
callback, conf, new RpcRetryingCallerFactory(conf));
|
new RpcRetryingCallerFactory(conf), false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public MyAsyncProcess(
|
||||||
|
HConnection hc, Configuration conf, boolean useGlobalErrors) {
|
||||||
|
super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
|
||||||
|
new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())),
|
||||||
|
new RpcRetryingCallerFactory(conf), useGlobalErrors);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public <Res> AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows,
|
||||||
|
boolean atLeastOne, Callback<Res> callback, boolean needResults)
|
||||||
|
throws InterruptedIOException {
|
||||||
|
// We use results in tests to check things, so override to always save them.
|
||||||
|
return super.submit(DUMMY_TABLE, rows, atLeastOne, callback, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -144,7 +185,6 @@ public class TestAsyncProcess {
|
||||||
* Returns our async process.
|
* Returns our async process.
|
||||||
*/
|
*/
|
||||||
static class MyConnectionImpl extends HConnectionManager.HConnectionImplementation {
|
static class MyConnectionImpl extends HConnectionManager.HConnectionImplementation {
|
||||||
MyAsyncProcess<?> ap;
|
|
||||||
final AtomicInteger nbThreads = new AtomicInteger(0);
|
final AtomicInteger nbThreads = new AtomicInteger(0);
|
||||||
final static Configuration c = new Configuration();
|
final static Configuration c = new Configuration();
|
||||||
|
|
||||||
|
@ -160,15 +200,6 @@ public class TestAsyncProcess {
|
||||||
super(conf);
|
super(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected <R> AsyncProcess createAsyncProcess(TableName tableName,
|
|
||||||
ExecutorService pool,
|
|
||||||
AsyncProcess.AsyncProcessCallback<R> callback,
|
|
||||||
Configuration confn ) {
|
|
||||||
ap = new MyAsyncProcess<R>(this, callback, c, nbThreads);
|
|
||||||
return ap;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public HRegionLocation locateRegion(final TableName tableName,
|
public HRegionLocation locateRegion(final TableName tableName,
|
||||||
final byte[] row) {
|
final byte[] row) {
|
||||||
|
@ -207,55 +238,57 @@ public class TestAsyncProcess {
|
||||||
@Test
|
@Test
|
||||||
public void testSubmit() throws Exception {
|
public void testSubmit() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
HConnection hc = createHConnection();
|
||||||
AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
|
AsyncProcess ap = new MyAsyncProcess(hc, conf);
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
|
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertTrue(puts.isEmpty());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSubmitWithCB() throws Exception {
|
public void testSubmitWithCB() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
HConnection hc = createHConnection();
|
||||||
MyCB mcb = new MyCB();
|
final AtomicInteger updateCalled = new AtomicInteger(0);
|
||||||
AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
Batch.Callback<Object> cb = new Batch.Callback<Object>() {
|
||||||
|
public void update(byte[] region, byte[] row, Object result) {
|
||||||
|
updateCalled.incrementAndGet();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
AsyncProcess ap = new MyAsyncProcess(hc, conf);
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
|
|
||||||
ap.submit(puts, false);
|
final AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, cb, false);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertTrue(puts.isEmpty());
|
||||||
|
ars.waitUntilDone();
|
||||||
while (!(mcb.successCalled.get() == 1) && !ap.hasError()) {
|
Assert.assertEquals(updateCalled.get(), 1);
|
||||||
Thread.sleep(1);
|
|
||||||
}
|
|
||||||
Assert.assertEquals(mcb.successCalled.get(), 1);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSubmitBusyRegion() throws Exception {
|
public void testSubmitBusyRegion() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
HConnection hc = createHConnection();
|
||||||
AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
|
AsyncProcess ap = new MyAsyncProcess(hc, conf);
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
|
|
||||||
ap.incTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
|
ap.incTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.assertEquals(puts.size(), 1);
|
Assert.assertEquals(puts.size(), 1);
|
||||||
|
|
||||||
ap.decTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
|
ap.decTaskCounters(Arrays.asList(hri1.getRegionName()), sn);
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertEquals(0, puts.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSubmitBusyRegionServer() throws Exception {
|
public void testSubmitBusyRegionServer() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
HConnection hc = createHConnection();
|
||||||
AsyncProcess<Object> ap = new MyAsyncProcess<Object>(hc, null, conf);
|
AsyncProcess ap = new MyAsyncProcess(hc, conf);
|
||||||
|
|
||||||
ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
|
ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer));
|
||||||
|
|
||||||
|
@ -265,80 +298,44 @@ public class TestAsyncProcess {
|
||||||
puts.add(createPut(1, true)); // <== this one will make it, the region is already in
|
puts.add(createPut(1, true)); // <== this one will make it, the region is already in
|
||||||
puts.add(createPut(2, true)); // <== new region, but the rs is ok
|
puts.add(createPut(2, true)); // <== new region, but the rs is ok
|
||||||
|
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.assertEquals(" puts=" + puts, 1, puts.size());
|
Assert.assertEquals(" puts=" + puts, 1, puts.size());
|
||||||
|
|
||||||
ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1));
|
ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1));
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertTrue(puts.isEmpty());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFail() throws Exception {
|
public void testFail() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
|
||||||
MyCB mcb = new MyCB();
|
|
||||||
AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
Put p = createPut(1, false);
|
Put p = createPut(1, false);
|
||||||
puts.add(p);
|
puts.add(p);
|
||||||
|
|
||||||
ap.submit(puts, false);
|
AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertEquals(0, puts.size());
|
||||||
|
ars.waitUntilDone();
|
||||||
|
verifyResult(ars, false);
|
||||||
|
Assert.assertEquals(2L, ap.getRetriesRequested());
|
||||||
|
|
||||||
while (!ap.hasError()) {
|
Assert.assertEquals(1, ars.getErrors().exceptions.size());
|
||||||
Thread.sleep(1);
|
Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0),
|
||||||
|
failure.equals(ars.getErrors().exceptions.get(0)));
|
||||||
|
Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0),
|
||||||
|
failure.equals(ars.getErrors().exceptions.get(0)));
|
||||||
|
|
||||||
|
Assert.assertEquals(1, ars.getFailedOperations().size());
|
||||||
|
Assert.assertTrue("was: " + ars.getFailedOperations().get(0),
|
||||||
|
p.equals(ars.getFailedOperations().get(0)));
|
||||||
}
|
}
|
||||||
|
|
||||||
Assert.assertEquals(0, mcb.successCalled.get());
|
|
||||||
Assert.assertEquals(2, mcb.retriableFailure.get());
|
|
||||||
Assert.assertEquals(1, mcb.failureCalled.get());
|
|
||||||
|
|
||||||
Assert.assertEquals(1, ap.getErrors().exceptions.size());
|
|
||||||
Assert.assertTrue("was: " + ap.getErrors().exceptions.get(0),
|
|
||||||
failure.equals(ap.getErrors().exceptions.get(0)));
|
|
||||||
Assert.assertTrue("was: " + ap.getErrors().exceptions.get(0),
|
|
||||||
failure.equals(ap.getErrors().exceptions.get(0)));
|
|
||||||
|
|
||||||
Assert.assertEquals(1, ap.getFailedOperations().size());
|
|
||||||
Assert.assertTrue("was: " + ap.getFailedOperations().get(0),
|
|
||||||
p.equals(ap.getFailedOperations().get(0)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testWaitForNextTaskDone() throws IOException {
|
|
||||||
HConnection hc = createHConnection();
|
|
||||||
MyCB mcb = new MyCB();
|
|
||||||
final AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
ap.tasksSent.incrementAndGet();
|
|
||||||
|
|
||||||
final AtomicBoolean checkPoint = new AtomicBoolean(false);
|
|
||||||
final AtomicBoolean checkPoint2 = new AtomicBoolean(false);
|
|
||||||
|
|
||||||
Thread t = new Thread(){
|
|
||||||
@Override
|
|
||||||
public void run(){
|
|
||||||
Threads.sleep(1000);
|
|
||||||
Assert.assertFalse(checkPoint.get());
|
|
||||||
ap.tasksDone.incrementAndGet();
|
|
||||||
checkPoint2.set(true);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
t.start();
|
|
||||||
ap.waitForNextTaskDone(0);
|
|
||||||
checkPoint.set(true);
|
|
||||||
while (!checkPoint2.get()){
|
|
||||||
Threads.sleep(1);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSubmitTrue() throws IOException {
|
public void testSubmitTrue() throws IOException {
|
||||||
HConnection hc = createHConnection();
|
final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
|
||||||
MyCB mcb = new MyCB();
|
ap.tasksInProgress.incrementAndGet();
|
||||||
final AsyncProcess<Object> ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
ap.tasksSent.incrementAndGet();
|
|
||||||
final AtomicInteger ai = new AtomicInteger(1);
|
final AtomicInteger ai = new AtomicInteger(1);
|
||||||
ap.taskCounterPerRegion.put(hri1.getRegionName(), ai);
|
ap.taskCounterPerRegion.put(hri1.getRegionName(), ai);
|
||||||
|
|
||||||
|
@ -349,9 +346,9 @@ public class TestAsyncProcess {
|
||||||
@Override
|
@Override
|
||||||
public void run(){
|
public void run(){
|
||||||
Threads.sleep(1000);
|
Threads.sleep(1000);
|
||||||
Assert.assertFalse(checkPoint.get());
|
Assert.assertFalse(checkPoint.get()); // TODO: this is timing-dependent
|
||||||
ai.decrementAndGet();
|
ai.decrementAndGet();
|
||||||
ap.tasksDone.incrementAndGet();
|
ap.tasksInProgress.decrementAndGet();
|
||||||
checkPoint2.set(true);
|
checkPoint2.set(true);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -360,12 +357,12 @@ public class TestAsyncProcess {
|
||||||
Put p = createPut(1, true);
|
Put p = createPut(1, true);
|
||||||
puts.add(p);
|
puts.add(p);
|
||||||
|
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.assertFalse(puts.isEmpty());
|
Assert.assertFalse(puts.isEmpty());
|
||||||
|
|
||||||
t.start();
|
t.start();
|
||||||
|
|
||||||
ap.submit(puts, true);
|
ap.submit(DUMMY_TABLE, puts, true, null, false);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertTrue(puts.isEmpty());
|
||||||
|
|
||||||
checkPoint.set(true);
|
checkPoint.set(true);
|
||||||
|
@ -376,71 +373,50 @@ public class TestAsyncProcess {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFailAndSuccess() throws Exception {
|
public void testFailAndSuccess() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
|
||||||
MyCB mcb = new MyCB();
|
|
||||||
AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
puts.add(createPut(1, false));
|
puts.add(createPut(1, false));
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
|
|
||||||
ap.submit(puts, false);
|
AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
|
||||||
Assert.assertTrue(puts.isEmpty());
|
Assert.assertTrue(puts.isEmpty());
|
||||||
|
ars.waitUntilDone();
|
||||||
long cutoff = System.currentTimeMillis() + 60000;
|
verifyResult(ars, false, true, true);
|
||||||
while (!ap.hasError() && System.currentTimeMillis() < cutoff) {
|
Assert.assertEquals(2, ap.getRetriesRequested());
|
||||||
Thread.sleep(1);
|
Assert.assertEquals(1, ars.getErrors().actions.size());
|
||||||
}
|
|
||||||
Assert.assertTrue(ap.hasError());
|
|
||||||
ap.waitUntilDone();
|
|
||||||
|
|
||||||
Assert.assertEquals(mcb.successCalled.get(), 2);
|
|
||||||
Assert.assertEquals(mcb.retriableFailure.get(), 2);
|
|
||||||
Assert.assertEquals(mcb.failureCalled.get(), 1);
|
|
||||||
|
|
||||||
Assert.assertEquals(1, ap.getErrors().actions.size());
|
|
||||||
|
|
||||||
|
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
ap.submit(puts, false);
|
// Wait for AP to be free. While ars might have the result, ap counters are decreased later.
|
||||||
Assert.assertTrue(puts.isEmpty());
|
ap.waitUntilDone();
|
||||||
|
ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
|
||||||
while (mcb.successCalled.get() != 3) {
|
Assert.assertEquals(0, puts.size());
|
||||||
Thread.sleep(1);
|
ars.waitUntilDone();
|
||||||
}
|
Assert.assertEquals(2, ap.getRetriesRequested());
|
||||||
Assert.assertEquals(mcb.retriableFailure.get(), 2);
|
verifyResult(ars, true);
|
||||||
Assert.assertEquals(mcb.failureCalled.get(), 1);
|
|
||||||
|
|
||||||
ap.clearErrors();
|
|
||||||
Assert.assertTrue(ap.getErrors().actions.isEmpty());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFlush() throws Exception {
|
public void testFlush() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
|
||||||
MyCB mcb = new MyCB();
|
|
||||||
AsyncProcess ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
puts.add(createPut(1, false));
|
puts.add(createPut(1, false));
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
|
|
||||||
ap.submit(puts, false);
|
AsyncRequestFuture ars = ap.submit(DUMMY_TABLE, puts, false, null, true);
|
||||||
ap.waitUntilDone();
|
ars.waitUntilDone();
|
||||||
|
verifyResult(ars, false, true, true);
|
||||||
|
Assert.assertEquals(2, ap.getRetriesRequested());
|
||||||
|
|
||||||
Assert.assertEquals(mcb.successCalled.get(), 2);
|
Assert.assertEquals(1, ars.getFailedOperations().size());
|
||||||
Assert.assertEquals(mcb.retriableFailure.get(), 2);
|
|
||||||
Assert.assertEquals(mcb.failureCalled.get(), 1);
|
|
||||||
|
|
||||||
Assert.assertEquals(1, ap.getFailedOperations().size());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMaxTask() throws Exception {
|
public void testMaxTask() throws Exception {
|
||||||
HConnection hc = createHConnection();
|
final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false);
|
||||||
final AsyncProcess ap = new MyAsyncProcess<Object>(hc, null, conf);
|
|
||||||
|
|
||||||
for (int i = 0; i < 1000; i++) {
|
for (int i = 0; i < 1000; i++) {
|
||||||
ap.incTaskCounters(Arrays.asList("dummy".getBytes()), sn);
|
ap.incTaskCounters(Arrays.asList("dummy".getBytes()), sn);
|
||||||
|
@ -461,7 +437,7 @@ public class TestAsyncProcess {
|
||||||
t.start();
|
t.start();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ap.submit(puts, false);
|
ap.submit(DUMMY_TABLE, puts, false, null, false);
|
||||||
Assert.fail("We should have been interrupted.");
|
Assert.fail("We should have been interrupted.");
|
||||||
} catch (InterruptedIOException expected) {
|
} catch (InterruptedIOException expected) {
|
||||||
}
|
}
|
||||||
|
@ -471,7 +447,7 @@ public class TestAsyncProcess {
|
||||||
Thread t2 = new Thread() {
|
Thread t2 = new Thread() {
|
||||||
public void run() {
|
public void run() {
|
||||||
Threads.sleep(sleepTime);
|
Threads.sleep(sleepTime);
|
||||||
while (ap.tasksDone.get() > 0) {
|
while (ap.tasksInProgress.get() > 0) {
|
||||||
ap.decTaskCounters(Arrays.asList("dummy".getBytes()), sn);
|
ap.decTaskCounters(Arrays.asList("dummy".getBytes()), sn);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -479,39 +455,13 @@ public class TestAsyncProcess {
|
||||||
t2.start();
|
t2.start();
|
||||||
|
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
ap.submit(new ArrayList<Row>(), false);
|
ap.submit(DUMMY_TABLE, new ArrayList<Row>(), false, null, false);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
|
|
||||||
//Adds 100 to secure us against approximate timing.
|
//Adds 100 to secure us against approximate timing.
|
||||||
Assert.assertTrue(start + 100L + sleepTime > end);
|
Assert.assertTrue(start + 100L + sleepTime > end);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private class MyCB implements AsyncProcess.AsyncProcessCallback<Object> {
|
|
||||||
private final AtomicInteger successCalled = new AtomicInteger(0);
|
|
||||||
private final AtomicInteger failureCalled = new AtomicInteger(0);
|
|
||||||
private final AtomicInteger retriableFailure = new AtomicInteger(0);
|
|
||||||
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void success(int originalIndex, byte[] region, Row row, Object o) {
|
|
||||||
successCalled.incrementAndGet();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean failure(int originalIndex, Row row, Throwable t) {
|
|
||||||
failureCalled.incrementAndGet();
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean retriableFailure(int originalIndex, Row row, Throwable exception) {
|
|
||||||
// We retry once only.
|
|
||||||
return (retriableFailure.incrementAndGet() < 2);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
private static HConnection createHConnection() throws IOException {
|
private static HConnection createHConnection() throws IOException {
|
||||||
HConnection hc = Mockito.mock(HConnection.class);
|
HConnection hc = Mockito.mock(HConnection.class);
|
||||||
|
|
||||||
|
@ -535,14 +485,17 @@ public class TestAsyncProcess {
|
||||||
Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
|
Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE),
|
||||||
Mockito.eq(FAILS))).thenReturn(loc2);
|
Mockito.eq(FAILS))).thenReturn(loc2);
|
||||||
|
|
||||||
|
NonceGenerator ng = Mockito.mock(NonceGenerator.class);
|
||||||
|
Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE);
|
||||||
|
Mockito.when(hc.getNonceGenerator()).thenReturn(ng);
|
||||||
|
|
||||||
return hc;
|
return hc;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHTablePutSuccess() throws Exception {
|
public void testHTablePutSuccess() throws Exception {
|
||||||
HTable ht = Mockito.mock(HTable.class);
|
HTable ht = Mockito.mock(HTable.class);
|
||||||
HConnection hc = createHConnection();
|
ht.ap = new MyAsyncProcess(createHConnection(), conf, true);
|
||||||
ht.ap = new MyAsyncProcess<Object>(hc, null, conf);
|
|
||||||
|
|
||||||
Put put = createPut(1, true);
|
Put put = createPut(1, true);
|
||||||
|
|
||||||
|
@ -553,9 +506,8 @@ public class TestAsyncProcess {
|
||||||
|
|
||||||
private void doHTableFailedPut(boolean bufferOn) throws Exception {
|
private void doHTableFailedPut(boolean bufferOn) throws Exception {
|
||||||
HTable ht = new HTable();
|
HTable ht = new HTable();
|
||||||
HConnection hc = createHConnection();
|
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true);
|
||||||
MyCB mcb = new MyCB(); // This allows to have some hints on what's going on.
|
ht.ap = ap;
|
||||||
ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
ht.setAutoFlush(true, true);
|
ht.setAutoFlush(true, true);
|
||||||
if (bufferOn) {
|
if (bufferOn) {
|
||||||
ht.setWriteBufferSize(1024L * 1024L);
|
ht.setWriteBufferSize(1024L * 1024L);
|
||||||
|
@ -575,9 +527,15 @@ public class TestAsyncProcess {
|
||||||
} catch (RetriesExhaustedException expected) {
|
} catch (RetriesExhaustedException expected) {
|
||||||
}
|
}
|
||||||
Assert.assertEquals(0L, ht.currentWriteBufferSize);
|
Assert.assertEquals(0L, ht.currentWriteBufferSize);
|
||||||
Assert.assertEquals(0, mcb.successCalled.get());
|
// The table should have sent one request, maybe after multiple attempts
|
||||||
Assert.assertEquals(2, mcb.retriableFailure.get());
|
AsyncRequestFuture ars = null;
|
||||||
Assert.assertEquals(1, mcb.failureCalled.get());
|
for (AsyncRequestFuture someReqs : ap.allReqs) {
|
||||||
|
if (someReqs.getResults().length == 0) continue;
|
||||||
|
Assert.assertTrue(ars == null);
|
||||||
|
ars = someReqs;
|
||||||
|
}
|
||||||
|
Assert.assertTrue(ars != null);
|
||||||
|
verifyResult(ars, false);
|
||||||
|
|
||||||
// This should not raise any exception, puts have been 'received' before by the catch.
|
// This should not raise any exception, puts have been 'received' before by the catch.
|
||||||
ht.close();
|
ht.close();
|
||||||
|
@ -589,23 +547,22 @@ public class TestAsyncProcess {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void doHTableFailedPutWithoutBuffer() throws Exception {
|
public void testHTableFailedPutWithoutBuffer() throws Exception {
|
||||||
doHTableFailedPut(false);
|
doHTableFailedPut(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHTableFailedPutAndNewPut() throws Exception {
|
public void testHTableFailedPutAndNewPut() throws Exception {
|
||||||
HTable ht = new HTable();
|
HTable ht = new HTable();
|
||||||
HConnection hc = createHConnection();
|
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true);
|
||||||
MyCB mcb = new MyCB(); // This allows to have some hints on what's going on.
|
ht.ap = ap;
|
||||||
ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
ht.setAutoFlush(false, true);
|
ht.setAutoFlush(false, true);
|
||||||
ht.setWriteBufferSize(0);
|
ht.setWriteBufferSize(0);
|
||||||
|
|
||||||
Put p = createPut(1, false);
|
Put p = createPut(1, false);
|
||||||
ht.put(p);
|
ht.put(p);
|
||||||
|
|
||||||
ht.ap.waitUntilDone(); // Let's do all the retries.
|
ap.waitUntilDone(); // Let's do all the retries.
|
||||||
|
|
||||||
// We're testing that we're behaving as we were behaving in 0.94: sending exceptions in the
|
// We're testing that we're behaving as we were behaving in 0.94: sending exceptions in the
|
||||||
// doPut if it fails.
|
// doPut if it fails.
|
||||||
|
@ -626,14 +583,13 @@ public class TestAsyncProcess {
|
||||||
@Test
|
@Test
|
||||||
public void testWithNoClearOnFail() throws IOException {
|
public void testWithNoClearOnFail() throws IOException {
|
||||||
HTable ht = new HTable();
|
HTable ht = new HTable();
|
||||||
HConnection hc = createHConnection();
|
ht.ap = new MyAsyncProcess(createHConnection(), conf, true);
|
||||||
MyCB mcb = new MyCB();
|
|
||||||
ht.ap = new MyAsyncProcess<Object>(hc, mcb, conf);
|
|
||||||
ht.setAutoFlush(false, false);
|
ht.setAutoFlush(false, false);
|
||||||
|
|
||||||
Put p = createPut(1, false);
|
Put p = createPut(1, false);
|
||||||
ht.put(p);
|
ht.put(p);
|
||||||
Assert.assertEquals(0, ht.writeAsyncBuffer.size());
|
Assert.assertEquals(0, ht.writeAsyncBuffer.size());
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ht.flushCommits();
|
ht.flushCommits();
|
||||||
} catch (RetriesExhaustedWithDetailsException expected) {
|
} catch (RetriesExhaustedWithDetailsException expected) {
|
||||||
|
@ -651,6 +607,7 @@ public class TestAsyncProcess {
|
||||||
public void testBatch() throws IOException, InterruptedException {
|
public void testBatch() throws IOException, InterruptedException {
|
||||||
HTable ht = new HTable();
|
HTable ht = new HTable();
|
||||||
ht.connection = new MyConnectionImpl();
|
ht.connection = new MyConnectionImpl();
|
||||||
|
ht.multiAp = new MyAsyncProcess(ht.connection, conf, false);
|
||||||
|
|
||||||
List<Put> puts = new ArrayList<Put>();
|
List<Put> puts = new ArrayList<Put>();
|
||||||
puts.add(createPut(1, true));
|
puts.add(createPut(1, true));
|
||||||
|
@ -686,10 +643,9 @@ public class TestAsyncProcess {
|
||||||
// set default writeBufferSize
|
// set default writeBufferSize
|
||||||
ht.setWriteBufferSize(configuration.getLong("hbase.client.write.buffer", 2097152));
|
ht.setWriteBufferSize(configuration.getLong("hbase.client.write.buffer", 2097152));
|
||||||
|
|
||||||
MyConnectionImpl mci = new MyConnectionImpl(configuration);
|
ht.connection = new MyConnectionImpl(configuration);
|
||||||
ht.connection = mci;
|
MyAsyncProcess ap = new MyAsyncProcess(ht.connection, conf, true);
|
||||||
ht.ap = new MyAsyncProcess<Object>(mci, null, configuration);
|
ht.ap = ap;
|
||||||
|
|
||||||
|
|
||||||
Assert.assertNotNull(ht.ap.createServerErrorTracker());
|
Assert.assertNotNull(ht.ap.createServerErrorTracker());
|
||||||
Assert.assertTrue(ht.ap.serverTrackerTimeout > 200);
|
Assert.assertTrue(ht.ap.serverTrackerTimeout > 200);
|
||||||
|
@ -705,7 +661,7 @@ public class TestAsyncProcess {
|
||||||
} catch (RetriesExhaustedWithDetailsException expected) {
|
} catch (RetriesExhaustedWithDetailsException expected) {
|
||||||
}
|
}
|
||||||
// Checking that the ErrorsServers came into play and didn't make us stop immediately
|
// Checking that the ErrorsServers came into play and didn't make us stop immediately
|
||||||
Assert.assertEquals(ht.ap.tasksSent.get(), 3);
|
Assert.assertEquals(2, ap.getRetriesRequested());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -730,11 +686,13 @@ public class TestAsyncProcess {
|
||||||
HTable ht = new HTable();
|
HTable ht = new HTable();
|
||||||
MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
|
MyConnectionImpl2 con = new MyConnectionImpl2(hrls);
|
||||||
ht.connection = con;
|
ht.connection = con;
|
||||||
|
MyAsyncProcess ap = new MyAsyncProcess(con, conf, con.nbThreads);
|
||||||
|
ht.multiAp = ap;
|
||||||
|
|
||||||
ht.batch(gets);
|
ht.batch(gets);
|
||||||
|
|
||||||
Assert.assertEquals(con.ap.nbActions.get(), NB_REGS);
|
Assert.assertEquals(ap.nbActions.get(), NB_REGS);
|
||||||
Assert.assertEquals("1 multi response per server", 2, con.ap.nbMultiResponse.get());
|
Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get());
|
||||||
Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
|
Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
|
||||||
|
|
||||||
int nbReg = 0;
|
int nbReg = 0;
|
||||||
|
@ -744,6 +702,13 @@ public class TestAsyncProcess {
|
||||||
Assert.assertEquals("nbReg=" + nbReg, nbReg, NB_REGS);
|
Assert.assertEquals("nbReg=" + nbReg, nbReg, NB_REGS);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void verifyResult(AsyncRequestFuture ars, boolean... expected) {
|
||||||
|
Object[] actual = ars.getResults();
|
||||||
|
Assert.assertEquals(expected.length, actual.length);
|
||||||
|
for (int i = 0; i < expected.length; ++i) {
|
||||||
|
Assert.assertEquals(expected[i], !(actual[i] instanceof Throwable));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param regCnt the region: 1 to 3.
|
* @param regCnt the region: 1 to 3.
|
||||||
|
|
|
@ -63,6 +63,7 @@ import com.google.protobuf.ServiceException;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
public class CoprocessorHConnection implements HConnection {
|
public class CoprocessorHConnection implements HConnection {
|
||||||
|
private static final NonceGenerator ng = new HConnectionManager.NoNonceGenerator();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create an unmanaged {@link HConnection} based on the environment in which we are running the
|
* Create an unmanaged {@link HConnection} based on the environment in which we are running the
|
||||||
|
@ -388,6 +389,11 @@ public class CoprocessorHConnection implements HConnection {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NonceGenerator getNonceGenerator() {
|
public NonceGenerator getNonceGenerator() {
|
||||||
return null; // don't use nonces for coprocessor connection
|
return ng; // don't use nonces for coprocessor connection
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AsyncProcess getAsyncProcess() {
|
||||||
|
return delegate.getAsyncProcess();
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -117,6 +117,10 @@ public class HConnectionTestingUtility {
|
||||||
Mockito.when(c.getClient(Mockito.any(ServerName.class))).
|
Mockito.when(c.getClient(Mockito.any(ServerName.class))).
|
||||||
thenReturn(client);
|
thenReturn(client);
|
||||||
}
|
}
|
||||||
|
NonceGenerator ng = Mockito.mock(NonceGenerator.class);
|
||||||
|
Mockito.when(c.getNonceGenerator()).thenReturn(ng);
|
||||||
|
Mockito.when(c.getAsyncProcess()).thenReturn(new AsyncProcess(
|
||||||
|
c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false));
|
||||||
return c;
|
return c;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -62,8 +62,13 @@ import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
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.MultiResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
|
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.MutateResponse;
|
||||||
|
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.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
@ -73,6 +78,8 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
import com.google.protobuf.RpcController;
|
import com.google.protobuf.RpcController;
|
||||||
import com.google.protobuf.Service;
|
import com.google.protobuf.Service;
|
||||||
|
@ -105,6 +112,18 @@ public class TestCatalogJanitor {
|
||||||
} catch (ServiceException se) {
|
} catch (ServiceException se) {
|
||||||
throw ProtobufUtil.getRemoteException(se);
|
throw ProtobufUtil.getRemoteException(se);
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
|
Mockito.when(ri.multi(
|
||||||
|
(RpcController)Mockito.any(), (MultiRequest)Mockito.any())).
|
||||||
|
thenAnswer(new Answer<MultiResponse>() {
|
||||||
|
@Override
|
||||||
|
public MultiResponse answer(InvocationOnMock invocation) throws Throwable {
|
||||||
|
return buildMultiResponse( (MultiRequest)invocation.getArguments()[1]);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
} catch (ServiceException se) {
|
||||||
|
throw ProtobufUtil.getRemoteException(se);
|
||||||
|
}
|
||||||
// Mock an HConnection and a AdminProtocol implementation. Have the
|
// Mock an HConnection and a AdminProtocol implementation. Have the
|
||||||
// HConnection return the HRI. Have the HRI return a few mocked up responses
|
// HConnection return the HRI. Have the HRI return a few mocked up responses
|
||||||
// to make our test work.
|
// to make our test work.
|
||||||
|
@ -940,5 +959,23 @@ public class TestCatalogJanitor {
|
||||||
return htd;
|
return htd;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private MultiResponse buildMultiResponse(MultiRequest req) {
|
||||||
|
MultiResponse.Builder builder = MultiResponse.newBuilder();
|
||||||
|
RegionActionResult.Builder regionActionResultBuilder =
|
||||||
|
RegionActionResult.newBuilder();
|
||||||
|
ResultOrException.Builder roeBuilder = ResultOrException.newBuilder();
|
||||||
|
for (RegionAction regionAction: req.getRegionActionList()) {
|
||||||
|
regionActionResultBuilder.clear();
|
||||||
|
for (ClientProtos.Action action: regionAction.getActionList()) {
|
||||||
|
roeBuilder.clear();
|
||||||
|
roeBuilder.setResult(ClientProtos.Result.getDefaultInstance());
|
||||||
|
roeBuilder.setIndex(action.getIndex());
|
||||||
|
regionActionResultBuilder.addResultOrException(roeBuilder.build());
|
||||||
|
}
|
||||||
|
builder.addRegionActionResult(regionActionResultBuilder.build());
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue