diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 50a2a11..d113f67 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -19,45 +19,35 @@ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.AsyncProcess.RowChecker.ReturnCode; -import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows; +import org.apache.hadoop.hbase.client.RequestController.ReturnCode; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdge; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; /** * This class allows a continuous flow of requests. It's written to be compatible with a @@ -95,9 +85,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; *

*/ @InterfaceAudience.Private +@InterfaceStability.Evolving class AsyncProcess { private static final Log LOG = LogFactory.getLog(AsyncProcess.class); - protected static final AtomicLong COUNTER = new AtomicLong(); + private static final AtomicLong COUNTER = new AtomicLong(); public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget"; @@ -116,31 +107,6 @@ class AsyncProcess { */ public static final String LOG_DETAILS_FOR_BATCH_ERROR = "hbase.client.log.batcherrors.details"; - protected final int thresholdToLogUndoneTaskDetails; - private static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = - "hbase.client.threshold.log.details"; - private static final int DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = 10; - private static final int THRESHOLD_TO_LOG_REGION_DETAILS = 2; - - /** - * The maximum size of single RegionServer. - */ - public static final String HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = "hbase.client.max.perrequest.heapsize"; - - /** - * Default value of #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE - */ - public static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304; - - /** - * The maximum size of submit. - */ - public static final String HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = "hbase.client.max.submit.heapsize"; - /** - * Default value of #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE - */ - public static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE; - /** * Return value from a submit that didn't contain any requests. */ @@ -173,64 +139,42 @@ class AsyncProcess { }; // TODO: many of the fields should be made private - protected final long id; - - protected final ClusterConnection connection; - protected final RpcRetryingCallerFactory rpcCallerFactory; - protected final RpcControllerFactory rpcFactory; - protected final BatchErrors globalErrors; - protected final ExecutorService pool; - - protected final AtomicLong tasksInProgress = new AtomicLong(0); - protected final ConcurrentMap taskCounterPerRegion = - new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); - protected final ConcurrentMap taskCounterPerServer = - new ConcurrentHashMap(); - // Start configuration settings. - protected final int startLogErrorsCnt; + final long id; - /** - * The number of tasks simultaneously executed on the cluster. - */ - protected final int maxTotalConcurrentTasks; + final ClusterConnection connection; + private final RpcRetryingCallerFactory rpcCallerFactory; + final RpcControllerFactory rpcFactory; + final BatchErrors globalErrors; - /** - * The max heap size of all tasks simultaneously executed on a server. - */ - protected final long maxHeapSizePerRequest; - protected final long maxHeapSizeSubmit; - /** - * The number of tasks we run in parallel on a single region. - * With 1 (the default) , we ensure that the ordering of the queries is respected: we don't start - * a set of operations on a region before the previous one is done. As well, this limits - * the pressure we put on the region server. - */ - protected final int maxConcurrentTasksPerRegion; + // Start configuration settings. + final int startLogErrorsCnt; - /** - * The number of task simultaneously executed on a single region server. - */ - protected final int maxConcurrentTasksPerServer; - protected final long pause; - protected final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified - protected int numTries; - protected int serverTrackerTimeout; - protected int rpcTimeout; - protected int operationTimeout; - protected long primaryCallTimeoutMicroseconds; + final long pause; + final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified + final int numTries; + @VisibleForTesting + int serverTrackerTimeout; + final long primaryCallTimeoutMicroseconds; /** Whether to log details for batch errors */ - protected final boolean logBatchErrorDetails; + final boolean logBatchErrorDetails; // End configuration settings. - public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool, + /** + * The traffic control for requests. + */ + @VisibleForTesting + final RequestController requestController; + public static final String LOG_DETAILS_PERIOD = "hbase.client.log.detail.period.ms"; + private static final int DEFAULT_LOG_DETAILS_PERIOD = 10000; + private final int periodToLogger; + AsyncProcess(ClusterConnection hc, Configuration conf, RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors, - RpcControllerFactory rpcFactory, int rpcTimeout, int operationTimeout) { + RpcControllerFactory rpcFactory) { if (hc == null) { throw new IllegalArgumentException("ClusterConnection cannot be null."); } this.connection = hc; - this.pool = pool; this.globalErrors = useGlobalErrors ? new BatchErrors() : null; this.id = COUNTER.incrementAndGet(); @@ -249,42 +193,10 @@ class AsyncProcess { // how many times we could try in total, one more than retry number this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER) + 1; - this.rpcTimeout = rpcTimeout; - this.operationTimeout = operationTimeout; this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000); - - this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS); - this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS); - this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS); - this.maxHeapSizePerRequest = conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, - DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); - this.maxHeapSizeSubmit = conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE); this.startLogErrorsCnt = conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT); - - if (this.maxTotalConcurrentTasks <= 0) { - throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks); - } - if (this.maxConcurrentTasksPerServer <= 0) { - throw new IllegalArgumentException("maxConcurrentTasksPerServer=" + - maxConcurrentTasksPerServer); - } - if (this.maxConcurrentTasksPerRegion <= 0) { - throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" + - maxConcurrentTasksPerRegion); - } - if (this.maxHeapSizePerRequest <= 0) { - throw new IllegalArgumentException("maxHeapSizePerServer=" + - maxHeapSizePerRequest); - } - - if (this.maxHeapSizeSubmit <= 0) { - throw new IllegalArgumentException("maxHeapSizeSubmit=" + - maxHeapSizeSubmit); - } + this.periodToLogger = conf.getInt(LOG_DETAILS_PERIOD, DEFAULT_LOG_DETAILS_PERIOD); // Server tracker allows us to do faster, and yet useful (hopefully), retries. // However, if we are too useful, we might fail very quickly due to retry count limit. // To avoid this, we are going to cheat for now (see HBASE-7659), and calculate maximum @@ -301,43 +213,30 @@ class AsyncProcess { this.rpcFactory = rpcFactory; this.logBatchErrorDetails = conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false); - this.thresholdToLogUndoneTaskDetails = - conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS, - DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS); + this.requestController = RequestControllerFactory.create(conf); } /** - * @return pool if non null, otherwise returns this.pool if non null, otherwise throws - * RuntimeException + * The submitted task may be not accomplished at all if there are too many running tasks or + * other limits. + * @param The class to cast the result + * @param task The setting and data + * @return AsyncRequestFuture */ - protected ExecutorService getPool(ExecutorService pool) { - if (pool != null) { - return pool; - } - if (this.pool != null) { - return this.pool; + public AsyncRequestFuture submit(AsyncProcessTask task) throws InterruptedIOException { + AsyncRequestFuture reqFuture = checkTask(task); + if (reqFuture != null) { + return reqFuture; + } + SubmittedRows submittedRows = task.getSubmittedRows() == null ? SubmittedRows.ALL : task.getSubmittedRows(); + switch (submittedRows) { + case ALL: + return submitAll(task); + case AT_LEAST_ONE: + return submit(task, true); + default: + return submit(task, false); } - throw new RuntimeException("Neither AsyncProcess nor request have ExecutorService"); - } - - /** - * See #submit(ExecutorService, TableName, RowAccess, boolean, Batch.Callback, boolean). - * Uses default ExecutorService for this AP (must have been created with one). - */ - public AsyncRequestFuture submit(TableName tableName, - final RowAccess rows, boolean atLeastOne, Batch.Callback callback, - boolean needResults) throws InterruptedIOException { - return submit(null, tableName, rows, atLeastOne, callback, needResults); - } - /** - * See {@link #submit(ExecutorService, TableName, RowAccess, boolean, Batch.Callback, boolean)}. - * Uses the {@link ListRowAccess} to wrap the {@link List}. - */ - public AsyncRequestFuture submit(ExecutorService pool, TableName tableName, - List rows, boolean atLeastOne, Batch.Callback callback, - boolean needResults) throws InterruptedIOException { - return submit(pool, tableName, new ListRowAccess(rows), atLeastOne, - callback, needResults); } /** @@ -351,14 +250,13 @@ class AsyncProcess { * @param needResults Whether results are needed, or can be discarded. * @param rows - the submitted row. Modified by the method: we remove the rows we took. * @param atLeastOne true if we should submit at least a subset. + * @param rpcTimeout timeout for each rpc request + * @param operationTimeout timeout for each call */ - public AsyncRequestFuture submit(ExecutorService pool, TableName tableName, - RowAccess rows, boolean atLeastOne, Batch.Callback callback, - boolean needResults) throws InterruptedIOException { - if (rows.isEmpty()) { - return NO_REQS_RESULT; - } - + private AsyncRequestFuture submit(AsyncProcessTask task, + boolean atLeastOne) throws InterruptedIOException { + TableName tableName = task.getTableName(); + RowAccess rows = task.getRowAccess(); Map actionsByServer = new HashMap(); List retainedActions = new ArrayList(rows.size()); @@ -369,11 +267,11 @@ class AsyncProcess { // Location errors that happen before we decide what requests to take. List locationErrors = null; List locationErrorRows = null; - RowCheckerHost checker = createRowCheckerHost(); + RequestController.Checker checker = requestController.newChecker(); boolean firstIter = true; do { // Wait until there is at least one slot for a new task. - waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1, tableName.getNameAsString()); + requestController.waitForFreeSlot(id, periodToLogger, getLogger(tableName, -1)); int posInList = -1; if (!firstIter) { checker.reset(); @@ -406,8 +304,7 @@ class AsyncProcess { it.remove(); break; // Backward compat: we stop considering actions on location error. } - long rowSize = (r instanceof Mutation) ? ((Mutation) r).heapSize() : 0; - ReturnCode code = checker.canTakeOperation(loc, rowSize); + ReturnCode code = checker.canTakeRow(loc, r); if (code == ReturnCode.END) { break; } @@ -426,29 +323,14 @@ class AsyncProcess { if (retainedActions.isEmpty()) return NO_REQS_RESULT; - return submitMultiActions(tableName, retainedActions, nonceGroup, callback, null, needResults, - locationErrors, locationErrorRows, actionsByServer, pool); + return submitMultiActions(task, retainedActions, nonceGroup, + locationErrors, locationErrorRows, actionsByServer); } - private RowCheckerHost createRowCheckerHost() { - return new RowCheckerHost(Arrays.asList( - new TaskCountChecker(maxTotalConcurrentTasks, - maxConcurrentTasksPerServer, - maxConcurrentTasksPerRegion, - tasksInProgress, - taskCounterPerServer, - taskCounterPerRegion) - , new RequestSizeChecker(maxHeapSizePerRequest) - , new SubmittedSizeChecker(maxHeapSizeSubmit) - )); - } - AsyncRequestFuture submitMultiActions(TableName tableName, - List retainedActions, long nonceGroup, Batch.Callback callback, - Object[] results, boolean needResults, List locationErrors, - List locationErrorRows, Map actionsByServer, - ExecutorService pool) { - AsyncRequestFutureImpl ars = createAsyncRequestFuture( - tableName, retainedActions, nonceGroup, pool, callback, results, needResults, null, -1); + AsyncRequestFuture submitMultiActions(AsyncProcessTask task, + List retainedActions, long nonceGroup, List locationErrors, + List locationErrorRows, Map actionsByServer) { + AsyncRequestFutureImpl ars = createAsyncRequestFuture(task, retainedActions, nonceGroup); // Add location errors if any if (locationErrors != null) { for (int i = 0; i < locationErrors.size(); ++i) { @@ -462,14 +344,6 @@ class AsyncProcess { return ars; } - public void setRpcTimeout(int rpcTimeout) { - this.rpcTimeout = rpcTimeout; - } - - public void setOperationTimeout(int operationTimeout) { - this.operationTimeout = operationTimeout; - } - /** * Helper that is used when grouping the actions per region server. * @@ -493,10 +367,6 @@ class AsyncProcess { multiAction.add(regionName, action); } - public AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName, - List rows, Batch.Callback callback, Object[] results) { - return submitAll(pool, tableName, rows, callback, results, null, -1); - } /** * Submit immediately the list of rows, whatever the server status. Kept for backward * compatibility: it allows to be used with the batch interface that return an array of objects. @@ -506,11 +376,11 @@ class AsyncProcess { * @param rows the list of rows. * @param callback the callback. * @param results Optional array to return the results thru; backward compat. - * @param rpcTimeout rpc timeout for this batch, set -1 if want to use current setting. + * @param rpcTimeout timeout for each rpc request + * @param operationTimeout timeout for each call */ - public AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName, - List rows, Batch.Callback callback, Object[] results, - CancellableRegionServerCallable callable, int rpcTimeout) { + private AsyncRequestFuture submitAll(AsyncProcessTask task) { + RowAccess rows = task.getRowAccess(); List actions = new ArrayList(rows.size()); // The position will be used by the processBatch to match the object array returned. @@ -528,93 +398,78 @@ class AsyncProcess { setNonce(ng, r, action); actions.add(action); } - AsyncRequestFutureImpl ars = createAsyncRequestFuture( - tableName, actions, ng.getNonceGroup(), getPool(pool), callback, results, results != null, - callable, rpcTimeout); + AsyncRequestFutureImpl ars = createAsyncRequestFuture(task, actions, ng.getNonceGroup()); ars.groupAndSendMultiAction(actions, 1); return ars; } + private AsyncRequestFuture checkTask(AsyncProcessTask task) { + if (task.getRowAccess() == null || task.getRowAccess().isEmpty()) { + return NO_REQS_RESULT; + } + Objects.requireNonNull(task.getPool(), "The pool can't be NULL"); + checkOperationTimeout(task.getOperationTimeout()); + checkRpcTimeout(task.getRpcTimeout()); + return null; + } + private void setNonce(NonceGenerator ng, Row r, Action action) { if (!(r instanceof Append) && !(r instanceof Increment)) return; action.setNonce(ng.newNonce()); // Action handles NO_NONCE, so it's ok if ng is disabled. } - protected AsyncRequestFutureImpl createAsyncRequestFuture( - TableName tableName, List actions, long nonceGroup, ExecutorService pool, - Batch.Callback callback, Object[] results, boolean needResults, - CancellableRegionServerCallable callable, int rpcTimeout) { - return new AsyncRequestFutureImpl( - tableName, actions, nonceGroup, getPool(pool), needResults, - results, callback, callable, operationTimeout, - rpcTimeout > 0 ? rpcTimeout : this.rpcTimeout, this); + private int checkTimeout(String name, int timeout) { + if (timeout < 0) { + throw new RuntimeException("The " + name + " must be bigger than zero," + + "current value is" + timeout); + } + return timeout; + } + private int checkOperationTimeout(int operationTimeout) { + return checkTimeout("operation timeout", operationTimeout); + } + + private int checkRpcTimeout(int rpcTimeout) { + return checkTimeout("rpc timeout", rpcTimeout); + } + + @VisibleForTesting + AsyncRequestFutureImpl createAsyncRequestFuture( + AsyncProcessTask task, List actions, long nonceGroup) { + return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this); } /** Wait until the async does not have more than max tasks in progress. */ - protected void waitForMaximumCurrentTasks(int max, String tableName) + protected void waitForMaximumCurrentTasks(int max, TableName tableName) throws InterruptedIOException { - waitForMaximumCurrentTasks(max, tasksInProgress, id, tableName); + requestController.waitForMaximumCurrentTasks(max, id, periodToLogger, + getLogger(tableName, max)); } - // Break out this method so testable - @VisibleForTesting - void waitForMaximumCurrentTasks(int max, final AtomicLong tasksInProgress, final long id, - String tableName) throws InterruptedIOException { - long lastLog = EnvironmentEdgeManager.currentTime(); - long currentInProgress, oldInProgress = Long.MAX_VALUE; - while ((currentInProgress = tasksInProgress.get()) > max) { - if (oldInProgress != currentInProgress) { // Wait for in progress to change. - long now = EnvironmentEdgeManager.currentTime(); - if (now > lastLog + 10000) { - lastLog = now; - LOG.info("#" + id + ", waiting for some tasks to finish. Expected max=" - + max + ", tasksInProgress=" + currentInProgress + - " hasError=" + hasError() + tableName == null ? "" : ", tableName=" + tableName); - if (currentInProgress <= thresholdToLogUndoneTaskDetails) { - logDetailsOfUndoneTasks(currentInProgress); - } - } - } - oldInProgress = currentInProgress; - try { - synchronized (tasksInProgress) { - if (tasksInProgress.get() == oldInProgress) { - tasksInProgress.wait(10); - } - } - } catch (InterruptedException e) { - throw new InterruptedIOException("#" + id + ", interrupted." + - " currentNumberOfTask=" + currentInProgress); - } - } + private Consumer getLogger(TableName tableName, long max) { + return (currentInProgress) -> { + LOG.info("#" + id + (max < 0 ? ", waiting for any free slot" + : ", waiting for some tasks to finish. Expected max=" + + max) + ", tasksInProgress=" + currentInProgress + + " hasError=" + hasError() + tableName == null ? "" : ", tableName=" + tableName); + }; } - void logDetailsOfUndoneTasks(long taskInProgress) { - ArrayList servers = new ArrayList(); - for (Map.Entry entry : taskCounterPerServer.entrySet()) { - if (entry.getValue().get() > 0) { - servers.add(entry.getKey()); - } - } - LOG.info("Left over " + taskInProgress + " task(s) are processed on server(s): " + servers); - if (taskInProgress <= THRESHOLD_TO_LOG_REGION_DETAILS) { - ArrayList regions = new ArrayList(); - for (Map.Entry entry : taskCounterPerRegion.entrySet()) { - if (entry.getValue().get() > 0) { - regions.add(Bytes.toString(entry.getKey())); - } - } - LOG.info("Regions against which left over task(s) are processed: " + regions); - } + void incTaskCounters(Collection regions, ServerName sn) { + requestController.incTaskCounters(regions, sn); } + + void decTaskCounters(Collection regions, ServerName sn) { + requestController.decTaskCounters(regions, sn); + } /** * Only used w/useGlobalErrors ctor argument, for HTable backward compat. * @return Whether there were any errors in any request since the last time * {@link #waitForAllPreviousOpsAndReset(List, String)} was called, or AP was created. */ public boolean hasError() { - return globalErrors.hasErrors(); + return globalErrors != null && globalErrors.hasErrors(); } /** @@ -628,9 +483,9 @@ class AsyncProcess { * was called, or AP was created. */ public RetriesExhaustedWithDetailsException waitForAllPreviousOpsAndReset( - List failedRows, String tableName) throws InterruptedIOException { + List failedRows, TableName tableName) throws InterruptedIOException { waitForMaximumCurrentTasks(0, tableName); - if (!globalErrors.hasErrors()) { + if (globalErrors == null || !globalErrors.hasErrors()) { return null; } if (failedRows != null) { @@ -642,41 +497,12 @@ class AsyncProcess { } /** - * increment the tasks counters for a given set of regions. MT safe. - */ - protected void incTaskCounters(Collection regions, ServerName sn) { - tasksInProgress.incrementAndGet(); - - computeIfAbsent(taskCounterPerServer, sn, AtomicInteger::new).incrementAndGet(); - - for (byte[] regBytes : regions) { - computeIfAbsent(taskCounterPerRegion, regBytes, AtomicInteger::new).incrementAndGet(); - } - } - - /** - * Decrements the counters for a given region and the region server. MT Safe. - */ - protected void decTaskCounters(Collection regions, ServerName sn) { - for (byte[] regBytes : regions) { - AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes); - regionCnt.decrementAndGet(); - } - - taskCounterPerServer.get(sn).decrementAndGet(); - tasksInProgress.decrementAndGet(); - synchronized (tasksInProgress) { - tasksInProgress.notifyAll(); - } - } - - /** * Create a caller. Isolated to be easily overridden in the tests. */ @VisibleForTesting protected RpcRetryingCaller createCaller( CancellableRegionServerCallable callable, int rpcTimeout) { - return rpcCallerFactory. newCaller(rpcTimeout); + return rpcCallerFactory. newCaller(checkRpcTimeout(rpcTimeout)); } @@ -687,7 +513,7 @@ class AsyncProcess { * We may benefit from connection-wide tracking of server errors. * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection */ - protected ConnectionImplementation.ServerErrorTracker createServerErrorTracker() { + ConnectionImplementation.ServerErrorTracker createServerErrorTracker() { return new ConnectionImplementation.ServerErrorTracker( this.serverTrackerTimeout, this.numTries); } @@ -696,283 +522,4 @@ class AsyncProcess { return (row instanceof Get) && (((Get)row).getConsistency() == Consistency.TIMELINE); } - /** - * Collect all advices from checkers and make the final decision. - */ - @VisibleForTesting - static class RowCheckerHost { - private final List checkers; - private boolean isEnd = false; - RowCheckerHost(final List checkers) { - this.checkers = checkers; - } - void reset() throws InterruptedIOException { - isEnd = false; - InterruptedIOException e = null; - for (RowChecker checker : checkers) { - try { - checker.reset(); - } catch (InterruptedIOException ex) { - e = ex; - } - } - if (e != null) { - throw e; - } - } - ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { - if (isEnd) { - return ReturnCode.END; - } - ReturnCode code = ReturnCode.INCLUDE; - for (RowChecker checker : checkers) { - switch (checker.canTakeOperation(loc, rowSize)) { - case END: - isEnd = true; - code = ReturnCode.END; - break; - case SKIP: - code = ReturnCode.SKIP; - break; - case INCLUDE: - default: - break; - } - if (code == ReturnCode.END) { - break; - } - } - for (RowChecker checker : checkers) { - checker.notifyFinal(code, loc, rowSize); - } - return code; - } - } - - /** - * Provide a way to control the flow of rows iteration. - */ - // Visible for Testing. Adding @VisibleForTesting here doesn't work for some reason. - interface RowChecker { - enum ReturnCode { - /** - * Accept current row. - */ - INCLUDE, - /** - * Skip current row. - */ - SKIP, - /** - * No more row can be included. - */ - END - }; - ReturnCode canTakeOperation(HRegionLocation loc, long rowSize); - /** - * Add the final ReturnCode to the checker. - * The ReturnCode may be reversed, so the checker need the final decision to update - * the inner state. - */ - void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize); - /** - * Reset the inner state. - */ - void reset() throws InterruptedIOException ; - } - - /** - * limit the heapsize of total submitted data. - * Reduce the limit of heapsize for submitting quickly - * if there is no running task. - */ - @VisibleForTesting - static class SubmittedSizeChecker implements RowChecker { - private final long maxHeapSizeSubmit; - private long heapSize = 0; - SubmittedSizeChecker(final long maxHeapSizeSubmit) { - this.maxHeapSizeSubmit = maxHeapSizeSubmit; - } - @Override - public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { - if (heapSize >= maxHeapSizeSubmit) { - return ReturnCode.END; - } - return ReturnCode.INCLUDE; - } - - @Override - public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) { - if (code == ReturnCode.INCLUDE) { - heapSize += rowSize; - } - } - - @Override - public void reset() { - heapSize = 0; - } - } - /** - * limit the max number of tasks in an AsyncProcess. - */ - @VisibleForTesting - static class TaskCountChecker implements RowChecker { - private static final long MAX_WAITING_TIME = 1000; //ms - private final Set regionsIncluded = new HashSet<>(); - private final Set serversIncluded = new HashSet<>(); - private final int maxConcurrentTasksPerRegion; - private final int maxTotalConcurrentTasks; - private final int maxConcurrentTasksPerServer; - private final Map taskCounterPerRegion; - private final Map taskCounterPerServer; - private final Set busyRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR); - private final AtomicLong tasksInProgress; - TaskCountChecker(final int maxTotalConcurrentTasks, - final int maxConcurrentTasksPerServer, - final int maxConcurrentTasksPerRegion, - final AtomicLong tasksInProgress, - final Map taskCounterPerServer, - final Map taskCounterPerRegion) { - this.maxTotalConcurrentTasks = maxTotalConcurrentTasks; - this.maxConcurrentTasksPerRegion = maxConcurrentTasksPerRegion; - this.maxConcurrentTasksPerServer = maxConcurrentTasksPerServer; - this.taskCounterPerRegion = taskCounterPerRegion; - this.taskCounterPerServer = taskCounterPerServer; - this.tasksInProgress = tasksInProgress; - } - @Override - public void reset() throws InterruptedIOException { - // prevent the busy-waiting - waitForRegion(); - regionsIncluded.clear(); - serversIncluded.clear(); - busyRegions.clear(); - } - private void waitForRegion() throws InterruptedIOException { - if (busyRegions.isEmpty()) { - return; - } - EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate(); - final long start = ee.currentTime(); - while ((ee.currentTime() - start) <= MAX_WAITING_TIME) { - for (byte[] region : busyRegions) { - AtomicInteger count = taskCounterPerRegion.get(region); - if (count == null || count.get() < maxConcurrentTasksPerRegion) { - return; - } - } - try { - synchronized (tasksInProgress) { - tasksInProgress.wait(10); - } - } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted." + - " tasksInProgress=" + tasksInProgress); - } - } - } - /** - * 1) check the regions is allowed. - * 2) check the concurrent tasks for regions. - * 3) check the total concurrent tasks. - * 4) check the concurrent tasks for server. - * @param loc - * @param rowSize - * @return - */ - @Override - public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { - - HRegionInfo regionInfo = loc.getRegionInfo(); - if (regionsIncluded.contains(regionInfo)) { - // We already know what to do with this region. - return ReturnCode.INCLUDE; - } - AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName()); - if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) { - // Too many tasks on this region already. - return ReturnCode.SKIP; - } - int newServers = serversIncluded.size() - + (serversIncluded.contains(loc.getServerName()) ? 0 : 1); - if ((newServers + tasksInProgress.get()) > maxTotalConcurrentTasks) { - // Too many tasks. - return ReturnCode.SKIP; - } - AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName()); - if (serverCnt != null && serverCnt.get() >= maxConcurrentTasksPerServer) { - // Too many tasks for this individual server - return ReturnCode.SKIP; - } - return ReturnCode.INCLUDE; - } - - @Override - public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) { - if (code == ReturnCode.INCLUDE) { - regionsIncluded.add(loc.getRegionInfo()); - serversIncluded.add(loc.getServerName()); - } - busyRegions.add(loc.getRegionInfo().getRegionName()); - } - } - - /** - * limit the request size for each regionserver. - */ - @VisibleForTesting - static class RequestSizeChecker implements RowChecker { - private final long maxHeapSizePerRequest; - private final Map serverRequestSizes = new HashMap<>(); - RequestSizeChecker(final long maxHeapSizePerRequest) { - this.maxHeapSizePerRequest = maxHeapSizePerRequest; - } - @Override - public void reset() { - serverRequestSizes.clear(); - } - @Override - public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { - // Is it ok for limit of request size? - long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName()) ? - serverRequestSizes.get(loc.getServerName()) : 0L; - // accept at least one request - if (currentRequestSize == 0 || currentRequestSize + rowSize <= maxHeapSizePerRequest) { - return ReturnCode.INCLUDE; - } - return ReturnCode.SKIP; - } - - @Override - public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) { - if (code == ReturnCode.INCLUDE) { - long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName()) ? - serverRequestSizes.get(loc.getServerName()) : 0L; - serverRequestSizes.put(loc.getServerName(), currentRequestSize + rowSize); - } - } - } - - public static class ListRowAccess implements RowAccess { - private final List data; - ListRowAccess(final List data) { - this.data = data; - } - - @Override - public int size() { - return data.size(); - } - - @Override - public boolean isEmpty() { - return data.isEmpty(); - } - - @Override - public Iterator iterator() { - return data.iterator(); - } - } } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java new file mode 100644 index 0000000..eda1db2 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java @@ -0,0 +1,229 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.client; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.coprocessor.Batch; + +/** + * Contains the attributes of a task which will be executed + * by {@link org.apache.hadoop.hbase.client.AsyncProcess}. + * The attributes will be validated by AsyncProcess. + * It's intended for advanced client applications. + * @param The type of response from server-side + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class AsyncProcessTask { + /** + * The number of processed rows. + * The AsyncProcess has traffic control which may reject some rows. + */ + public enum SubmittedRows { + ALL, + AT_LEAST_ONE, + NORMAL + } + public static Builder newBuilder(final Batch.Callback callback) { + return new Builder<>(callback); + } + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + + private ExecutorService pool; + private TableName tableName; + private RowAccess rows; + private SubmittedRows submittedRows = SubmittedRows.ALL; + private Batch.Callback callback; + private boolean needResults; + private int rpcTimeout; + private int operationTimeout; + private CancellableRegionServerCallable callable; + private Object[] results; + + private Builder() { + } + + private Builder(Batch.Callback callback) { + this.callback = callback; + } + + Builder setResults(Object[] results) { + this.results = results; + if (results != null && results.length != 0) { + setNeedResults(true); + } + return this; + } + + public Builder setPool(ExecutorService pool) { + this.pool = pool; + return this; + } + + public Builder setRpcTimeout(int rpcTimeout) { + this.rpcTimeout = rpcTimeout; + return this; + } + + public Builder setOperationTimeout(int operationTimeout) { + this.operationTimeout = operationTimeout; + return this; + } + + public Builder setTableName(TableName tableName) { + this.tableName = tableName; + return this; + } + + public Builder setRowAccess(List rows) { + this.rows = new ListRowAccess<>(rows); + return this; + } + + public Builder setRowAccess(RowAccess rows) { + this.rows = rows; + return this; + } + + public Builder setSubmittedRows(SubmittedRows submittedRows) { + this.submittedRows = submittedRows; + return this; + } + + public Builder setNeedResults(boolean needResults) { + this.needResults = needResults; + return this; + } + + Builder setCallable(CancellableRegionServerCallable callable) { + this.callable = callable; + return this; + } + + public AsyncProcessTask build() { + return new AsyncProcessTask<>(pool, tableName, rows, submittedRows, + callback, callable, needResults, rpcTimeout, operationTimeout, results); + } + } + private final ExecutorService pool; + private final TableName tableName; + private final RowAccess rows; + private final SubmittedRows submittedRows; + private final Batch.Callback callback; + private final CancellableRegionServerCallable callable; + private final boolean needResults; + private final int rpcTimeout; + private final int operationTimeout; + private final Object[] results; + AsyncProcessTask(AsyncProcessTask task) { + this(task.getPool(), task.getTableName(), task.getRowAccess(), + task.getSubmittedRows(), task.getCallback(), task.getCallable(), + task.getNeedResults(), task.getRpcTimeout(), task.getOperationTimeout(), + task.getResults()); + } + AsyncProcessTask(ExecutorService pool, TableName tableName, + RowAccess rows, SubmittedRows size, Batch.Callback callback, + CancellableRegionServerCallable callable, boolean needResults, + int rpcTimeout, int operationTimeout, Object[] results) { + this.pool = pool; + this.tableName = tableName; + this.rows = rows; + this.submittedRows = size; + this.callback = callback; + this.callable = callable; + this.needResults = needResults; + this.rpcTimeout = rpcTimeout; + this.operationTimeout = operationTimeout; + this.results = results; + } + + public int getOperationTimeout() { + return operationTimeout; + } + + public ExecutorService getPool() { + return pool; + } + + public TableName getTableName() { + return tableName; + } + + public RowAccess getRowAccess() { + return rows; + } + + public SubmittedRows getSubmittedRows() { + return submittedRows; + } + + public Batch.Callback getCallback() { + return callback; + } + + CancellableRegionServerCallable getCallable() { + return callable; + } + + Object[] getResults() { + return results; + } + + public boolean getNeedResults() { + return needResults; + } + + public int getRpcTimeout() { + return rpcTimeout; + } + + static class ListRowAccess implements RowAccess { + + private final List data; + + ListRowAccess(final List data) { + this.data = data; + } + + @Override + public int size() { + return data.size(); + } + + @Override + public boolean isEmpty() { + return data.isEmpty(); + } + + @Override + public Iterator iterator() { + return data.iterator(); + } + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java index d176ce1..036196e 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java @@ -300,11 +300,11 @@ class AsyncRequestFutureImpl implements AsyncRequestFuture { private final int[] replicaGetIndices; private final boolean hasAnyReplicaGets; private final long nonceGroup; - private CancellableRegionServerCallable currentCallable; - private int operationTimeout; - private int rpcTimeout; + private final CancellableRegionServerCallable currentCallable; + private final int operationTimeout; + private final int rpcTimeout; private final Map> heapSizesByServer = new HashMap<>(); - protected AsyncProcess asyncProcess; + private final AsyncProcess asyncProcess; /** * For {@link AsyncRequestFutureImpl#manageError(int, Row, Retry, Throwable, ServerName)}. Only @@ -339,32 +339,27 @@ class AsyncRequestFutureImpl implements AsyncRequestFuture { } } - - - public AsyncRequestFutureImpl(TableName tableName, List actions, long nonceGroup, - ExecutorService pool, boolean needResults, Object[] results, Batch.Callback callback, - CancellableRegionServerCallable callable, int operationTimeout, int rpcTimeout, - AsyncProcess asyncProcess) { - this.pool = pool; - this.callback = callback; + public AsyncRequestFutureImpl(AsyncProcessTask task, List actions, + long nonceGroup, AsyncProcess asyncProcess) { + this.pool = task.getPool(); + this.callback = task.getCallback(); this.nonceGroup = nonceGroup; - this.tableName = tableName; + this.tableName = task.getTableName(); this.actionsInProgress.set(actions.size()); - if (results != null) { - assert needResults; - if (results.length != actions.size()) { + if (task.getResults() == null) { + results = task.getNeedResults() ? new Object[actions.size()] : null; + } else { + if (task.getResults().length != actions.size()) { throw new AssertionError("results.length"); } - this.results = results; + this.results = task.getResults(); for (int i = 0; i != this.results.length; ++i) { results[i] = null; } - } else { - this.results = needResults ? new Object[actions.size()] : null; } List replicaGetIndices = null; boolean hasAnyReplicaGets = false; - if (needResults) { + if (results != null) { // Check to see if any requests might require replica calls. // We expect that many requests will consist of all or no multi-replica gets; in such // cases we would just use a boolean (hasAnyReplicaGets). If there's a mix, we will @@ -414,10 +409,10 @@ class AsyncRequestFutureImpl implements AsyncRequestFuture { this.errorsByServer = createServerErrorTracker(); this.errors = (asyncProcess.globalErrors != null) ? asyncProcess.globalErrors : new BatchErrors(); - this.operationTimeout = operationTimeout; - this.rpcTimeout = rpcTimeout; - this.currentCallable = callable; - if (callable == null) { + this.operationTimeout = task.getOperationTimeout(); + this.rpcTimeout = task.getRpcTimeout(); + this.currentCallable = task.getCallable(); + if (task.getCallable() == null) { tracker = new RetryingTimeTracker().start(); } } @@ -1246,9 +1241,6 @@ class AsyncRequestFutureImpl implements AsyncRequestFuture { lastLog = now; LOG.info("#" + asyncProcess.id + ", waiting for " + currentInProgress + " actions to finish on table: " + tableName); - if (currentInProgress <= asyncProcess.thresholdToLogUndoneTaskDetails) { - asyncProcess.logDetailsOfUndoneTasks(currentInProgress); - } } } synchronized (actionsInProgress) { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index 0085767..2a55de9 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -19,12 +19,9 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; // Needed for write rpc timeout import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.Collections; @@ -36,6 +33,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; /** *

@@ -67,61 +66,70 @@ public class BufferedMutatorImpl implements BufferedMutator { "hbase.client.bufferedmutator.classname"; private static final Log LOG = LogFactory.getLog(BufferedMutatorImpl.class); - + private final ExceptionListener listener; - protected ClusterConnection connection; // non-final so can be overridden in test private final TableName tableName; - private volatile Configuration conf; - - @VisibleForTesting - final ConcurrentLinkedQueue writeAsyncBuffer = new ConcurrentLinkedQueue(); - @VisibleForTesting - AtomicLong currentWriteBufferSize = new AtomicLong(0); + private final Configuration conf; + private final ConcurrentLinkedQueue writeAsyncBuffer = new ConcurrentLinkedQueue<>(); + private final AtomicLong currentWriteBufferSize = new AtomicLong(0); /** * Count the size of {@link BufferedMutatorImpl#writeAsyncBuffer}. * The {@link ConcurrentLinkedQueue#size()} is NOT a constant-time operation. */ - @VisibleForTesting - AtomicInteger undealtMutationCount = new AtomicInteger(0); - private long writeBufferSize; + private final AtomicInteger undealtMutationCount = new AtomicInteger(0); + private volatile long writeBufferSize; private final int maxKeyValueSize; - private boolean closed = false; private final ExecutorService pool; - private int writeRpcTimeout; // needed to pass in through AsyncProcess constructor - private int operationTimeout; + private final AtomicInteger rpcTimeout; + private final AtomicInteger operationTimeout; + private final boolean cleanupPoolOnClose; + private volatile boolean closed = false; + private final AsyncProcess ap; @VisibleForTesting - protected AsyncProcess ap; // non-final so can be overridden in test - - BufferedMutatorImpl(ClusterConnection conn, RpcRetryingCallerFactory rpcCallerFactory, - RpcControllerFactory rpcFactory, BufferedMutatorParams params) { + BufferedMutatorImpl(ClusterConnection conn, BufferedMutatorParams params, AsyncProcess ap) { if (conn == null || conn.isClosed()) { throw new IllegalArgumentException("Connection is null or closed."); } - this.tableName = params.getTableName(); - this.connection = conn; - this.conf = connection.getConfiguration(); - this.pool = params.getPool(); + this.conf = conn.getConfiguration(); this.listener = params.getListener(); - + if (params.getPool() == null) { + this.pool = HTable.getDefaultExecutor(conf); + cleanupPoolOnClose = true; + } else { + this.pool = params.getPool(); + cleanupPoolOnClose = false; + } ConnectionConfiguration tableConf = new ConnectionConfiguration(conf); this.writeBufferSize = params.getWriteBufferSize() != BufferedMutatorParams.UNSET ? params.getWriteBufferSize() : tableConf.getWriteBufferSize(); this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ? params.getMaxKeyValueSize() : tableConf.getMaxKeyValueSize(); - this.writeRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, - conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - this.operationTimeout = conn.getConfiguration().getInt( - HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - // puts need to track errors globally due to how the APIs currently work. - ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory, - writeRpcTimeout, operationTimeout); + this.rpcTimeout = new AtomicInteger(params.getRpcTimeout() != BufferedMutatorParams.UNSET ? + params.getRpcTimeout() : conn.getConnectionConfiguration().getWriteRpcTimeout()); + this.operationTimeout = new AtomicInteger(params.getOperationTimeout()!= BufferedMutatorParams.UNSET ? + params.getOperationTimeout() : conn.getConnectionConfiguration().getOperationTimeout()); + this.ap = ap; + } + BufferedMutatorImpl(ClusterConnection conn, RpcRetryingCallerFactory rpcCallerFactory, + RpcControllerFactory rpcFactory, BufferedMutatorParams params) { + this(conn, params, + // puts need to track errors globally due to how the APIs currently work. + new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, true, rpcFactory)); + } + + @VisibleForTesting + ExecutorService getPool() { + return pool; + } + + @VisibleForTesting + AsyncProcess getAsyncProcess() { + return ap; } @Override @@ -193,22 +201,22 @@ public class BufferedMutatorImpl implements BufferedMutator { // As we can have an operation in progress even if the buffer is empty, we call // backgroundFlushCommits at least one time. backgroundFlushCommits(true); - this.pool.shutdown(); - boolean terminated; - int loopCnt = 0; - do { - // wait until the pool has terminated - terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS); - loopCnt += 1; - if (loopCnt >= 10) { - LOG.warn("close() failed to terminate pool after 10 minutes. Abandoning pool."); - break; - } - } while (!terminated); - + if (cleanupPoolOnClose) { + this.pool.shutdown(); + boolean terminated; + int loopCnt = 0; + do { + // wait until the pool has terminated + terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS); + loopCnt += 1; + if (loopCnt >= 10) { + LOG.warn("close() failed to terminate pool after 10 minutes. Abandoning pool."); + break; + } + } while (!terminated); + } } catch (InterruptedException e) { LOG.warn("waitForTermination interrupted"); - } finally { this.closed = true; } @@ -239,8 +247,9 @@ public class BufferedMutatorImpl implements BufferedMutator { if (!synchronous) { QueueRowAccess taker = new QueueRowAccess(); + AsyncProcessTask task = wrapAsyncProcessTask(taker); try { - ap.submit(tableName, taker, true, null, false); + ap.submit(task); if (ap.hasError()) { LOG.debug(tableName + ": One or more of the operations have failed -" + " waiting for all operation in progress to finish (successfully or not)"); @@ -251,17 +260,17 @@ public class BufferedMutatorImpl implements BufferedMutator { } if (synchronous || ap.hasError()) { QueueRowAccess taker = new QueueRowAccess(); + AsyncProcessTask task = wrapAsyncProcessTask(taker); try { while (!taker.isEmpty()) { - ap.submit(tableName, taker, true, null, false); + ap.submit(task); taker.reset(); } } finally { taker.restoreRemainder(); } - RetriesExhaustedWithDetailsException error = - ap.waitForAllPreviousOpsAndReset(null, tableName.getNameAsString()); + ap.waitForAllPreviousOpsAndReset(null, tableName); if (error != null) { if (listener == null) { throw error; @@ -273,8 +282,38 @@ public class BufferedMutatorImpl implements BufferedMutator { } /** + * Reuse the AsyncProcessTask when calling {@link BufferedMutatorImpl#backgroundFlushCommits(boolean)}. + * @param taker access the inner buffer. + * @return An AsyncProcessTask which always returns the latest rpc and operation timeout. + */ + private AsyncProcessTask wrapAsyncProcessTask(QueueRowAccess taker) { + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(pool) + .setTableName(tableName) + .setRowAccess(taker) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.AT_LEAST_ONE) + .build(); + return new AsyncProcessTask(task) { + @Override + public int getRpcTimeout() { + return rpcTimeout.get(); + } + + @Override + public int getOperationTimeout() { + return operationTimeout.get(); + } + }; + } + /** * This is used for legacy purposes in {@link HTable#setWriteBufferSize(long)} only. This ought * not be called for production uses. + * If the new buffer size is smaller than the stored data, the {@link BufferedMutatorImpl#flush()} + * will be called. + * @param writeBufferSize The max size of internal buffer where data is stored. + * @throws org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException + * if an I/O error occurs and there are too many retries. + * @throws java.io.InterruptedIOException if the I/O task is interrupted. * @deprecated Going away when we drop public support for {@link HTable}. */ @Deprecated @@ -295,15 +334,23 @@ public class BufferedMutatorImpl implements BufferedMutator { } @Override - public void setRpcTimeout(int timeout) { - this.writeRpcTimeout = timeout; - ap.setRpcTimeout(timeout); + public void setRpcTimeout(int rpcTimeout) { + this.rpcTimeout.set(rpcTimeout); } @Override - public void setOperationTimeout(int timeout) { - this.operationTimeout = timeout; - ap.setOperationTimeout(operationTimeout); + public void setOperationTimeout(int operationTimeout) { + this.operationTimeout.set(operationTimeout); + } + + @VisibleForTesting + long getCurrentWriteBufferSize() { + return currentWriteBufferSize.get(); + } + + @VisibleForTesting + int size() { + return undealtMutationCount.get(); } private class QueueRowAccess implements RowAccess { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java index 17c69ec..9c901e2 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java @@ -39,7 +39,8 @@ public class BufferedMutatorParams implements Cloneable { private int maxKeyValueSize = UNSET; private ExecutorService pool = null; private String implementationClassName = null; - + private int rpcTimeout = UNSET; + private int operationTimeout = UNSET; private BufferedMutator.ExceptionListener listener = new BufferedMutator.ExceptionListener() { @Override public void onException(RetriesExhaustedWithDetailsException exception, @@ -61,6 +62,24 @@ public class BufferedMutatorParams implements Cloneable { return writeBufferSize; } + public BufferedMutatorParams rpcTimeout(final int rpcTimeout) { + this.rpcTimeout = rpcTimeout; + return this; + } + + public int getRpcTimeout() { + return rpcTimeout; + } + + public BufferedMutatorParams opertationTimeout(final int operationTimeout) { + this.operationTimeout = operationTimeout; + return this; + } + + public int getOperationTimeout() { + return operationTimeout; + } + /** * Override the write buffer size specified by the provided {@link Connection}'s * {@link org.apache.hadoop.conf.Configuration} instance, via the configuration key diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java index 35bebae..41f5baf 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java @@ -42,7 +42,8 @@ public class ConnectionConfiguration { private final int replicaCallTimeoutMicroSecondScan; private final int retries; private final int maxKeyValueSize; - + private final int readRpcTimeout; + private final int writeRpcTimeout; // toggle for async/sync prefetch private final boolean clientScannerAsyncPrefetch; @@ -80,6 +81,12 @@ public class ConnectionConfiguration { Scan.HBASE_CLIENT_SCANNER_ASYNC_PREFETCH, Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH); this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT); + + this.readRpcTimeout = conf.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, + conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + + this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, + conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); } /** @@ -99,6 +106,16 @@ public class ConnectionConfiguration { this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER; this.clientScannerAsyncPrefetch = Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH; this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT; + this.readRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT; + this.writeRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT; + } + + public int getReadRpcTimeout() { + return readRpcTimeout; + } + + public int getWriteRpcTimeout() { + return writeRpcTimeout; } public long getWriteBufferSize() { diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index ff939aa..880c5e7 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -243,7 +243,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats); this.backoffPolicy = ClientBackoffPolicyFactory.create(conf); - this.asyncProcess = createAsyncProcess(this.conf); + this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, false, rpcControllerFactory); if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) { this.metrics = new MetricsConnection(this); } else { @@ -1808,17 +1808,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { metaCache.clearCache(regionInfo); } - // For tests to override. - protected AsyncProcess createAsyncProcess(Configuration conf) { - // No default pool available. - int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - int operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - return new AsyncProcess(this, conf, batchPool, rpcCallerFactory, false, rpcControllerFactory, - rpcTimeout, operationTimeout); - } - @Override public AsyncProcess getAsyncProcess() { return asyncProcess; diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index dd11abf..f79a55e 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -103,27 +103,28 @@ import org.apache.hadoop.hbase.util.Threads; @InterfaceStability.Stable public class HTable implements Table { private static final Log LOG = LogFactory.getLog(HTable.class); - protected ClusterConnection connection; + private static final Consistency DEFAULT_CONSISTENCY = Consistency.STRONG; + private final ClusterConnection connection; private final TableName tableName; - private volatile Configuration configuration; - private ConnectionConfiguration connConfiguration; - protected BufferedMutatorImpl mutator; + private final Configuration configuration; + private final ConnectionConfiguration connConfiguration; + @VisibleForTesting + BufferedMutatorImpl mutator; private boolean closed = false; - protected int scannerCaching; - protected long scannerMaxResultSize; - private ExecutorService pool; // For Multi & Scan + private final int scannerCaching; + private final long scannerMaxResultSize; + private final ExecutorService pool; // For Multi & Scan private int operationTimeout; // global timeout for each blocking method with retrying rpc private int readRpcTimeout; // timeout for each read rpc request private int writeRpcTimeout; // timeout for each write rpc request private final boolean cleanupPoolOnClose; // shutdown the pool in close() - private final boolean cleanupConnectionOnClose; // close the connection in close() - private Consistency defaultConsistency = Consistency.STRONG; - private HRegionLocator locator; + private final HRegionLocator locator; /** The Async process for batch */ - protected AsyncProcess multiAp; - private RpcRetryingCallerFactory rpcCallerFactory; - private RpcControllerFactory rpcControllerFactory; + @VisibleForTesting + AsyncProcess multiAp; + private final RpcRetryingCallerFactory rpcCallerFactory; + private final RpcControllerFactory rpcControllerFactory; // Marked Private @since 1.0 @InterfaceAudience.Private @@ -167,22 +168,42 @@ public class HTable implements Table { throw new IllegalArgumentException("Given table name is null"); } this.tableName = tableName; - this.cleanupConnectionOnClose = false; this.connection = connection; this.configuration = connection.getConfiguration(); - this.connConfiguration = tableConfig; - this.pool = pool; + if (tableConfig == null) { + connConfiguration = new ConnectionConfiguration(configuration); + } else { + connConfiguration = tableConfig; + } if (pool == null) { this.pool = getDefaultExecutor(this.configuration); this.cleanupPoolOnClose = true; } else { + this.pool = pool; this.cleanupPoolOnClose = false; } + if (rpcCallerFactory == null) { + this.rpcCallerFactory = connection.getNewRpcRetryingCallerFactory(configuration); + } else { + this.rpcCallerFactory = rpcCallerFactory; + } - this.rpcCallerFactory = rpcCallerFactory; - this.rpcControllerFactory = rpcControllerFactory; + if (rpcControllerFactory == null) { + this.rpcControllerFactory = RpcControllerFactory.instantiate(configuration); + } else { + this.rpcControllerFactory = rpcControllerFactory; + } + + this.operationTimeout = tableName.isSystemTable() ? + connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout(); + this.readRpcTimeout = connConfiguration.getReadRpcTimeout(); + this.writeRpcTimeout = connConfiguration.getWriteRpcTimeout(); + this.scannerCaching = connConfiguration.getScannerCaching(); + this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize(); - this.finishSetup(); + // puts need to track errors globally due to how the APIs currently work. + multiAp = this.connection.getAsyncProcess(); + this.locator = new HRegionLocator(tableName, connection); } /** @@ -190,20 +211,23 @@ public class HTable implements Table { * @throws IOException */ @VisibleForTesting - protected HTable(ClusterConnection conn, BufferedMutatorParams params) throws IOException { + protected HTable(ClusterConnection conn, BufferedMutatorImpl mutator) throws IOException { connection = conn; - tableName = params.getTableName(); - connConfiguration = new ConnectionConfiguration(connection.getConfiguration()); + this.tableName = mutator.getName(); + this.configuration = connection.getConfiguration(); + connConfiguration = new ConnectionConfiguration(configuration); cleanupPoolOnClose = false; - cleanupConnectionOnClose = false; - // used from tests, don't trust the connection is real - this.mutator = new BufferedMutatorImpl(conn, null, null, params); - this.readRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, - conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - this.writeRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, - conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + this.mutator = mutator; + this.operationTimeout = tableName.isSystemTable() ? + connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout(); + this.readRpcTimeout = connConfiguration.getReadRpcTimeout(); + this.writeRpcTimeout = connConfiguration.getWriteRpcTimeout(); + this.scannerCaching = connConfiguration.getScannerCaching(); + this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize(); + this.rpcControllerFactory = null; + this.rpcCallerFactory = null; + this.pool = mutator.getPool(); + this.locator = null; } /** @@ -214,36 +238,6 @@ public class HTable implements Table { } /** - * setup this HTable's parameter based on the passed configuration - */ - private void finishSetup() throws IOException { - if (connConfiguration == null) { - connConfiguration = new ConnectionConfiguration(configuration); - } - - this.operationTimeout = tableName.isSystemTable() ? - connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout(); - this.readRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, - configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - this.writeRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, - configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - this.scannerCaching = connConfiguration.getScannerCaching(); - this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize(); - if (this.rpcCallerFactory == null) { - this.rpcCallerFactory = connection.getNewRpcRetryingCallerFactory(configuration); - } - if (this.rpcControllerFactory == null) { - this.rpcControllerFactory = RpcControllerFactory.instantiate(configuration); - } - - // puts need to track errors globally due to how the APIs currently work. - multiAp = this.connection.getAsyncProcess(); - this.locator = new HRegionLocator(getName(), connection); - } - - /** * {@inheritDoc} */ @Override @@ -423,7 +417,7 @@ public class HTable implements Table { get = ReflectionUtils.newInstance(get.getClass(), get); get.setCheckExistenceOnly(checkExistenceOnly); if (get.getConsistency() == null){ - get.setConsistency(defaultConsistency); + get.setConsistency(DEFAULT_CONSISTENCY); } } @@ -483,13 +477,37 @@ public class HTable implements Table { @Override public void batch(final List actions, final Object[] results) throws InterruptedException, IOException { - batch(actions, results, -1); + int rpcTimeout = writeRpcTimeout; + boolean hasRead = false; + boolean hasWrite = false; + for (Row action : actions) { + if (action instanceof Mutation) { + hasWrite = true; + } else { + hasRead = true; + } + if (hasRead && hasWrite) { + break; + } + } + if (hasRead && !hasWrite) { + rpcTimeout = readRpcTimeout; + } + batch(actions, results, rpcTimeout); } public void batch(final List actions, final Object[] results, int rpcTimeout) throws InterruptedException, IOException { - AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, actions, null, results, null, - rpcTimeout); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(pool) + .setTableName(tableName) + .setRowAccess(actions) + .setResults(results) + .setRpcTimeout(rpcTimeout) + .setOperationTimeout(operationTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { throw ars.getErrors(); @@ -509,8 +527,20 @@ public class HTable implements Table { public static void doBatchWithCallback(List actions, Object[] results, Callback callback, ClusterConnection connection, ExecutorService pool, TableName tableName) throws InterruptedIOException, RetriesExhaustedWithDetailsException { - AsyncRequestFuture ars = connection.getAsyncProcess().submitAll( - pool, tableName, actions, callback, results); + int operationTimeout = connection.getConnectionConfiguration().getOperationTimeout(); + int writeTimeout = connection.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, + connection.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + AsyncProcessTask task = AsyncProcessTask.newBuilder(callback) + .setPool(pool) + .setTableName(tableName) + .setRowAccess(actions) + .setResults(results) + .setOperationTimeout(operationTimeout) + .setRpcTimeout(writeTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = connection.getAsyncProcess().submit(task); ars.waitUntilDone(); if (ars.hasError()) { throw ars.getErrors(); @@ -536,8 +566,16 @@ public class HTable implements Table { } }; List rows = Collections.singletonList(delete); - AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rows, - null, null, callable, writeRpcTimeout); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(pool) + .setTableName(tableName) + .setRowAccess(rows) + .setCallable(callable) + .setRpcTimeout(writeRpcTimeout) + .setOperationTimeout(operationTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { throw ars.getErrors(); @@ -615,8 +653,16 @@ public class HTable implements Table { return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); } }; - AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(), - null, null, callable, writeRpcTimeout); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(pool) + .setTableName(tableName) + .setRowAccess(rm.getMutations()) + .setCallable(callable) + .setRpcTimeout(writeRpcTimeout) + .setOperationTimeout(operationTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { throw ars.getErrors(); @@ -795,8 +841,18 @@ public class HTable implements Table { }; List rows = Collections.singletonList(delete); Object[] results = new Object[1]; - AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rows, - null, results, callable, -1); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(pool) + .setTableName(tableName) + .setRowAccess(rows) + .setCallable(callable) + // TODO any better timeout? + .setRpcTimeout(readRpcTimeout + writeRpcTimeout) + .setOperationTimeout(operationTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .setResults(results) + .build(); + AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { throw ars.getErrors(); @@ -839,8 +895,18 @@ public class HTable implements Table { * It is excessive to send such a large array, but that is required by the framework right now * */ Object[] results = new Object[rm.getMutations().size()]; - AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(), - null, results, callable, -1); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(pool) + .setTableName(tableName) + .setRowAccess(rm.getMutations()) + .setResults(results) + .setCallable(callable) + // TODO any better timeout? + .setRpcTimeout(readRpcTimeout + writeRpcTimeout) + .setOperationTimeout(operationTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { throw ars.getErrors(); @@ -925,26 +991,27 @@ public class HTable implements Table { if (this.closed) { return; } - flushCommits(); - if (cleanupPoolOnClose) { - this.pool.shutdown(); - try { - boolean terminated = false; - do { - // wait until the pool has terminated - terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS); - } while (!terminated); - } catch (InterruptedException e) { - this.pool.shutdownNow(); - LOG.warn("waitForTermination interrupted"); + try { + flushCommits(); + if (mutator != null) { + mutator.close(); } - } - if (cleanupConnectionOnClose) { - if (this.connection != null) { - this.connection.close(); + if (cleanupPoolOnClose) { + this.pool.shutdown(); + try { + boolean terminated = false; + do { + // wait until the pool has terminated + terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS); + } while (!terminated); + } catch (InterruptedException e) { + this.pool.shutdownNow(); + LOG.warn("waitForTermination interrupted"); + } } + } finally { + this.closed = true; } - this.closed = true; } // validate for well-formedness @@ -1102,7 +1169,6 @@ public class HTable implements Table { if (mutator != null) { mutator.setOperationTimeout(operationTimeout); } - multiAp.setOperationTimeout(operationTimeout); } @Override @@ -1134,7 +1200,6 @@ public class HTable implements Table { if (mutator != null) { mutator.setRpcTimeout(writeRpcTimeout); } - multiAp.setRpcTimeout(writeRpcTimeout); } @Override @@ -1217,37 +1282,41 @@ public class HTable implements Table { Object[] results = new Object[execs.size()]; AsyncProcess asyncProcess = - new AsyncProcess(connection, configuration, pool, + new AsyncProcess(connection, configuration, RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()), - true, RpcControllerFactory.instantiate(configuration), readRpcTimeout, - operationTimeout); - - AsyncRequestFuture future = asyncProcess.submitAll(null, tableName, execs, - new Callback() { - @Override - public void update(byte[] region, byte[] row, - ClientProtos.CoprocessorServiceResult serviceResult) { - if (LOG.isTraceEnabled()) { - LOG.trace("Received result for endpoint " + methodDescriptor.getFullName() + - ": region=" + Bytes.toStringBinary(region) + - ", row=" + Bytes.toStringBinary(row) + - ", value=" + serviceResult.getValue().getValue()); - } - try { - Message.Builder builder = responsePrototype.newBuilderForType(); - org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builder, - serviceResult.getValue().getValue().toByteArray()); - callback.update(region, row, (R) builder.build()); - } catch (IOException e) { - LOG.error("Unexpected response type from endpoint " + methodDescriptor.getFullName(), - e); - callbackErrorExceptions.add(e); - callbackErrorActions.add(execsByRow.get(row)); - callbackErrorServers.add("null"); - } - } - }, results); - + true, RpcControllerFactory.instantiate(configuration)); + + Callback resultsCallback + = (byte[] region, byte[] row, ClientProtos.CoprocessorServiceResult serviceResult) -> { + if (LOG.isTraceEnabled()) { + LOG.trace("Received result for endpoint " + methodDescriptor.getFullName() + + ": region=" + Bytes.toStringBinary(region) + + ", row=" + Bytes.toStringBinary(row) + + ", value=" + serviceResult.getValue().getValue()); + } + try { + Message.Builder builder = responsePrototype.newBuilderForType(); + org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builder, + serviceResult.getValue().getValue().toByteArray()); + callback.update(region, row, (R) builder.build()); + } catch (IOException e) { + LOG.error("Unexpected response type from endpoint " + methodDescriptor.getFullName(), + e); + callbackErrorExceptions.add(e); + callbackErrorActions.add(execsByRow.get(row)); + callbackErrorServers.add("null"); + } + }; + AsyncProcessTask task = AsyncProcessTask.newBuilder(resultsCallback) + .setPool(pool) + .setTableName(tableName) + .setRowAccess(execs) + .setResults(results) + .setRpcTimeout(readRpcTimeout) + .setOperationTimeout(operationTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .build(); + AsyncRequestFuture future = asyncProcess.submit(task); future.waitUntilDone(); if (future.hasError()) { @@ -1270,10 +1339,10 @@ public class HTable implements Table { .pool(pool) .writeBufferSize(connConfiguration.getWriteBufferSize()) .maxKeyValueSize(connConfiguration.getMaxKeyValueSize()) + .opertationTimeout(operationTimeout) + .rpcTimeout(writeRpcTimeout) ); } - mutator.setRpcTimeout(writeRpcTimeout); - mutator.setOperationTimeout(operationTimeout); return mutator; } } \ No newline at end of file diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java index 8ff64bf..c03b969 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java @@ -443,7 +443,7 @@ public class HTableMultiplexer { private final AtomicInteger retryInQueue = new AtomicInteger(0); private final int writeRpcTimeout; // needed to pass in through AsyncProcess constructor private final int operationTimeout; - + private final ExecutorService pool; public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr, HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize, ExecutorService pool, ScheduledExecutorService executor) { @@ -457,10 +457,10 @@ public class HTableMultiplexer { HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory, - writeRpcTimeout, operationTimeout); + this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, false, rpcControllerFactory); this.executor = executor; this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000); + this.pool = pool; } protected LinkedBlockingQueue getQueue() { @@ -594,9 +594,14 @@ public class HTableMultiplexer { Map actionsByServer = Collections.singletonMap(server, actions); try { + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setResults(results) + .setPool(pool) + .setRpcTimeout(writeRpcTimeout) + .setOperationTimeout(operationTimeout) + .build(); AsyncRequestFuture arf = - ap.submitMultiActions(null, retainedActions, 0L, null, results, true, null, - null, actionsByServer, null); + ap.submitMultiActions(task, retainedActions, 0L, null, null, actionsByServer); arf.waitUntilDone(); if (arf.hasError()) { // We just log and ignore the exception here since failed Puts will be resubmit again. diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java new file mode 100644 index 0000000..7e9c968 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestController.java @@ -0,0 +1,125 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.client; + +import java.io.InterruptedIOException; +import java.util.Collection; +import java.util.function.Consumer; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * An interface for client request scheduling algorithm. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface RequestController { + + @InterfaceAudience.Public + @InterfaceStability.Evolving + public enum ReturnCode { + /** + * Accept current row. + */ + INCLUDE, + /** + * Skip current row. + */ + SKIP, + /** + * No more row can be included. + */ + END + } + + /** + * Picks up the valid data. + */ + @InterfaceAudience.Public + @InterfaceStability.Evolving + public interface Checker { + /** + * Checks the data whether it is valid to submit. + * @param loc the destination of data + * @param row the data to check + * @return describe the decision for the row + */ + ReturnCode canTakeRow(HRegionLocation loc, Row row); + + /** + * Reset the state of the scheduler when completing the iteration of rows. + * @throws InterruptedIOException some controller may wait + * for some busy region or RS to complete the undealt request. + */ + void reset() throws InterruptedIOException ; + } + + /** + * @return A new checker for evaluating a batch rows. + */ + Checker newChecker(); + + /** + * Increment the counter if we build a valid task. + * @param regions The destination of task + * @param sn The target server + */ + void incTaskCounters(Collection regions, ServerName sn); + + /** + * Decrement the counter if a task is accomplished. + * @param regions The destination of task + * @param sn The target server + */ + void decTaskCounters(Collection regions, ServerName sn); + + /** + * @return The number of running task. + */ + long getNumberOfTsksInProgress(); + + /** + * Waits for the running tasks to complete. + * If there are specified threshold and trigger, the implementation should + * wake up once in a while for checking the threshold and calling trigger. + * @param max This method will return if the number of running tasks is + * less than or equal to max. + * @param id the caller's id + * @param periodToTrigger The period to invoke the trigger. This value is a + * hint. The real period depends on the implementation. + * @param trigger The object to call periodically. + * @throws java.io.InterruptedIOException If the waiting is interrupted + */ + void waitForMaximumCurrentTasks(long max, long id, + int periodToTrigger, Consumer trigger) throws InterruptedIOException; + + /** + * Wait until there is at least one slot for a new task. + * @param id the caller's id + * @param periodToTrigger The period to invoke the trigger. This value is a + * hint. The real period depends on the implementation. + * @param trigger The object to call periodically. + * @throws java.io.InterruptedIOException If the waiting is interrupted + */ + void waitForFreeSlot(long id, int periodToTrigger, + Consumer trigger) throws InterruptedIOException; +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java new file mode 100644 index 0000000..7ed80f0 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestControllerFactory.java @@ -0,0 +1,44 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.ReflectionUtils; + +/** + * A factory class that constructs an {@link org.apache.hadoop.hbase.client.RequestController}. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class RequestControllerFactory { + public static final String REQUEST_CONTROLLER_IMPL_CONF_KEY = "hbase.client.request.controller.impl"; + /** + * Constructs a {@link org.apache.hadoop.hbase.client.RequestController}. + * @param conf The {@link Configuration} to use. + * @return A RequestController which is built according to the configuration. + */ + public static RequestController create(Configuration conf) { + Class clazz= conf.getClass(REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class, RequestController.class); + return ReflectionUtils.newInstance(clazz, conf); + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java index 788f1a4..85fd590 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RowAccess.java @@ -30,8 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; */ @InterfaceAudience.Public @InterfaceStability.Evolving -@VisibleForTesting -interface RowAccess extends Iterable { +public interface RowAccess extends Iterable { /** * @return true if there are no elements. */ diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java new file mode 100644 index 0000000..473f264 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java @@ -0,0 +1,519 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import com.google.common.annotations.VisibleForTesting; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; +import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent; +import org.apache.hadoop.hbase.util.EnvironmentEdge; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; + +/** + * Holds back the request if the submitted size or number has reached the + * threshold. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +class SimpleRequestController implements RequestController { + private static final Log LOG = LogFactory.getLog(SimpleRequestController.class); + /** + * The maximum size of single RegionServer. + */ + public static final String HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = "hbase.client.max.perrequest.heapsize"; + + /** + * Default value of #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE + */ + @VisibleForTesting + static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304; + + /** + * The maximum size of submit. + */ + public static final String HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = "hbase.client.max.submit.heapsize"; + /** + * Default value of #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE + */ + @VisibleForTesting + static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE; + @VisibleForTesting + final AtomicLong tasksInProgress = new AtomicLong(0); + @VisibleForTesting + final ConcurrentMap taskCounterPerRegion + = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR); + @VisibleForTesting + final ConcurrentMap taskCounterPerServer = new ConcurrentHashMap<>(); + /** + * The number of tasks simultaneously executed on the cluster. + */ + private final int maxTotalConcurrentTasks; + + /** + * The max heap size of all tasks simultaneously executed on a server. + */ + private final long maxHeapSizePerRequest; + private final long maxHeapSizeSubmit; + /** + * The number of tasks we run in parallel on a single region. With 1 (the + * default) , we ensure that the ordering of the queries is respected: we + * don't start a set of operations on a region before the previous one is + * done. As well, this limits the pressure we put on the region server. + */ + @VisibleForTesting + final int maxConcurrentTasksPerRegion; + + /** + * The number of task simultaneously executed on a single region server. + */ + @VisibleForTesting + final int maxConcurrentTasksPerServer; + private final int thresholdToLogUndoneTaskDetails; + public static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = + "hbase.client.threshold.log.details"; + private static final int DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = 10; + public static final String THRESHOLD_TO_LOG_REGION_DETAILS = + "hbase.client.threshold.log.region.details"; + private static final int DEFAULT_THRESHOLD_TO_LOG_REGION_DETAILS = 2; + private final int thresholdToLogRegionDetails; + SimpleRequestController(final Configuration conf) { + this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS); + this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS); + this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS, + HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS); + this.maxHeapSizePerRequest = conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, + DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); + this.maxHeapSizeSubmit = conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE); + this.thresholdToLogUndoneTaskDetails = + conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS, + DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS); + this.thresholdToLogRegionDetails = + conf.getInt(THRESHOLD_TO_LOG_REGION_DETAILS, + DEFAULT_THRESHOLD_TO_LOG_REGION_DETAILS); + if (this.maxTotalConcurrentTasks <= 0) { + throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks); + } + if (this.maxConcurrentTasksPerServer <= 0) { + throw new IllegalArgumentException("maxConcurrentTasksPerServer=" + + maxConcurrentTasksPerServer); + } + if (this.maxConcurrentTasksPerRegion <= 0) { + throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" + + maxConcurrentTasksPerRegion); + } + if (this.maxHeapSizePerRequest <= 0) { + throw new IllegalArgumentException("maxHeapSizePerServer=" + + maxHeapSizePerRequest); + } + + if (this.maxHeapSizeSubmit <= 0) { + throw new IllegalArgumentException("maxHeapSizeSubmit=" + + maxHeapSizeSubmit); + } + } + + @VisibleForTesting + static Checker newChecker(List checkers) { + return new Checker() { + private boolean isEnd = false; + + @Override + public ReturnCode canTakeRow(HRegionLocation loc, Row row) { + if (isEnd) { + return ReturnCode.END; + } + long rowSize = (row instanceof Mutation) ? ((Mutation) row).heapSize() : 0; + ReturnCode code = ReturnCode.INCLUDE; + for (RowChecker checker : checkers) { + switch (checker.canTakeOperation(loc, rowSize)) { + case END: + isEnd = true; + code = ReturnCode.END; + break; + case SKIP: + code = ReturnCode.SKIP; + break; + case INCLUDE: + default: + break; + } + if (code == ReturnCode.END) { + break; + } + } + for (RowChecker checker : checkers) { + checker.notifyFinal(code, loc, rowSize); + } + return code; + } + + @Override + public void reset() throws InterruptedIOException { + isEnd = false; + InterruptedIOException e = null; + for (RowChecker checker : checkers) { + try { + checker.reset(); + } catch (InterruptedIOException ex) { + e = ex; + } + } + if (e != null) { + throw e; + } + } + }; + } + + @Override + public Checker newChecker() { + List checkers = new ArrayList<>(3); + checkers.add(new TaskCountChecker(maxTotalConcurrentTasks, + maxConcurrentTasksPerServer, + maxConcurrentTasksPerRegion, + tasksInProgress, + taskCounterPerServer, + taskCounterPerRegion)); + checkers.add(new RequestSizeChecker(maxHeapSizePerRequest)); + checkers.add(new SubmittedSizeChecker(maxHeapSizeSubmit)); + return newChecker(checkers); + } + + @Override + public void incTaskCounters(Collection regions, ServerName sn) { + tasksInProgress.incrementAndGet(); + + computeIfAbsent(taskCounterPerServer, sn, AtomicInteger::new).incrementAndGet(); + + regions.forEach((regBytes) + -> computeIfAbsent(taskCounterPerRegion, regBytes, AtomicInteger::new).incrementAndGet() + ); + } + + @Override + public void decTaskCounters(Collection regions, ServerName sn) { + regions.forEach(regBytes -> { + AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes); + regionCnt.decrementAndGet(); + }); + + taskCounterPerServer.get(sn).decrementAndGet(); + tasksInProgress.decrementAndGet(); + synchronized (tasksInProgress) { + tasksInProgress.notifyAll(); + } + } + + @Override + public long getNumberOfTsksInProgress() { + return tasksInProgress.get(); + } + + @Override + public void waitForMaximumCurrentTasks(long max, long id, + int periodToTrigger, Consumer trigger) throws InterruptedIOException { + assert max >= 0; + long lastLog = EnvironmentEdgeManager.currentTime(); + long currentInProgress, oldInProgress = Long.MAX_VALUE; + while ((currentInProgress = tasksInProgress.get()) > max) { + if (oldInProgress != currentInProgress) { // Wait for in progress to change. + long now = EnvironmentEdgeManager.currentTime(); + if (now > lastLog + periodToTrigger) { + lastLog = now; + if (trigger != null) { + trigger.accept(currentInProgress); + } + logDetailsOfUndoneTasks(currentInProgress); + } + } + oldInProgress = currentInProgress; + try { + synchronized (tasksInProgress) { + if (tasksInProgress.get() == oldInProgress) { + tasksInProgress.wait(10); + } + } + } catch (InterruptedException e) { + throw new InterruptedIOException("#" + id + ", interrupted." + + " currentNumberOfTask=" + currentInProgress); + } + } + } + + private void logDetailsOfUndoneTasks(long taskInProgress) { + if (taskInProgress <= thresholdToLogUndoneTaskDetails) { + ArrayList servers = new ArrayList<>(); + for (Map.Entry entry : taskCounterPerServer.entrySet()) { + if (entry.getValue().get() > 0) { + servers.add(entry.getKey()); + } + } + LOG.info("Left over " + taskInProgress + " task(s) are processed on server(s): " + servers); + } + + if (taskInProgress <= thresholdToLogRegionDetails) { + ArrayList regions = new ArrayList<>(); + for (Map.Entry entry : taskCounterPerRegion.entrySet()) { + if (entry.getValue().get() > 0) { + regions.add(Bytes.toString(entry.getKey())); + } + } + LOG.info("Regions against which left over task(s) are processed: " + regions); + } + } + + @Override + public void waitForFreeSlot(long id, int periodToTrigger, Consumer trigger) throws InterruptedIOException { + waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1, id, periodToTrigger, trigger); + } + + /** + * limit the heapsize of total submitted data. Reduce the limit of heapsize + * for submitting quickly if there is no running task. + */ + @VisibleForTesting + static class SubmittedSizeChecker implements RowChecker { + + private final long maxHeapSizeSubmit; + private long heapSize = 0; + + SubmittedSizeChecker(final long maxHeapSizeSubmit) { + this.maxHeapSizeSubmit = maxHeapSizeSubmit; + } + + @Override + public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { + if (heapSize >= maxHeapSizeSubmit) { + return ReturnCode.END; + } + return ReturnCode.INCLUDE; + } + + @Override + public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) { + if (code == ReturnCode.INCLUDE) { + heapSize += rowSize; + } + } + + @Override + public void reset() { + heapSize = 0; + } + } + + /** + * limit the max number of tasks in an AsyncProcess. + */ + @VisibleForTesting + static class TaskCountChecker implements RowChecker { + + private static final long MAX_WAITING_TIME = 1000; //ms + private final Set regionsIncluded = new HashSet<>(); + private final Set serversIncluded = new HashSet<>(); + private final int maxConcurrentTasksPerRegion; + private final int maxTotalConcurrentTasks; + private final int maxConcurrentTasksPerServer; + private final Map taskCounterPerRegion; + private final Map taskCounterPerServer; + private final Set busyRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR); + private final AtomicLong tasksInProgress; + + TaskCountChecker(final int maxTotalConcurrentTasks, + final int maxConcurrentTasksPerServer, + final int maxConcurrentTasksPerRegion, + final AtomicLong tasksInProgress, + final Map taskCounterPerServer, + final Map taskCounterPerRegion) { + this.maxTotalConcurrentTasks = maxTotalConcurrentTasks; + this.maxConcurrentTasksPerRegion = maxConcurrentTasksPerRegion; + this.maxConcurrentTasksPerServer = maxConcurrentTasksPerServer; + this.taskCounterPerRegion = taskCounterPerRegion; + this.taskCounterPerServer = taskCounterPerServer; + this.tasksInProgress = tasksInProgress; + } + + @Override + public void reset() throws InterruptedIOException { + // prevent the busy-waiting + waitForRegion(); + regionsIncluded.clear(); + serversIncluded.clear(); + busyRegions.clear(); + } + + private void waitForRegion() throws InterruptedIOException { + if (busyRegions.isEmpty()) { + return; + } + EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate(); + final long start = ee.currentTime(); + while ((ee.currentTime() - start) <= MAX_WAITING_TIME) { + for (byte[] region : busyRegions) { + AtomicInteger count = taskCounterPerRegion.get(region); + if (count == null || count.get() < maxConcurrentTasksPerRegion) { + return; + } + } + try { + synchronized (tasksInProgress) { + tasksInProgress.wait(10); + } + } catch (InterruptedException e) { + throw new InterruptedIOException("Interrupted." + + " tasksInProgress=" + tasksInProgress); + } + } + } + + /** + * 1) check the regions is allowed. 2) check the concurrent tasks for + * regions. 3) check the total concurrent tasks. 4) check the concurrent + * tasks for server. + * + * @param loc + * @param rowSize + * @return + */ + @Override + public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { + + HRegionInfo regionInfo = loc.getRegionInfo(); + if (regionsIncluded.contains(regionInfo)) { + // We already know what to do with this region. + return ReturnCode.INCLUDE; + } + AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName()); + if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) { + // Too many tasks on this region already. + return ReturnCode.SKIP; + } + int newServers = serversIncluded.size() + + (serversIncluded.contains(loc.getServerName()) ? 0 : 1); + if ((newServers + tasksInProgress.get()) > maxTotalConcurrentTasks) { + // Too many tasks. + return ReturnCode.SKIP; + } + AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName()); + if (serverCnt != null && serverCnt.get() >= maxConcurrentTasksPerServer) { + // Too many tasks for this individual server + return ReturnCode.SKIP; + } + return ReturnCode.INCLUDE; + } + + @Override + public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) { + if (code == ReturnCode.INCLUDE) { + regionsIncluded.add(loc.getRegionInfo()); + serversIncluded.add(loc.getServerName()); + } + busyRegions.add(loc.getRegionInfo().getRegionName()); + } + } + + /** + * limit the request size for each regionserver. + */ + @VisibleForTesting + static class RequestSizeChecker implements RowChecker { + + private final long maxHeapSizePerRequest; + private final Map serverRequestSizes = new HashMap<>(); + + RequestSizeChecker(final long maxHeapSizePerRequest) { + this.maxHeapSizePerRequest = maxHeapSizePerRequest; + } + + @Override + public void reset() { + serverRequestSizes.clear(); + } + + @Override + public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) { + // Is it ok for limit of request size? + long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName()) + ? serverRequestSizes.get(loc.getServerName()) : 0L; + // accept at least one request + if (currentRequestSize == 0 || currentRequestSize + rowSize <= maxHeapSizePerRequest) { + return ReturnCode.INCLUDE; + } + return ReturnCode.SKIP; + } + + @Override + public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) { + if (code == ReturnCode.INCLUDE) { + long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName()) + ? serverRequestSizes.get(loc.getServerName()) : 0L; + serverRequestSizes.put(loc.getServerName(), currentRequestSize + rowSize); + } + } + } + + /** + * Provide a way to control the flow of rows iteration. + */ + @VisibleForTesting + interface RowChecker { + + ReturnCode canTakeOperation(HRegionLocation loc, long rowSize); + + /** + * Add the final ReturnCode to the checker. The ReturnCode may be reversed, + * so the checker need the final decision to update the inner state. + * + * @param code The final decision + * @param loc the destination of data + * @param rowSize the data size + */ + void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize); + + /** + * Reset the inner state. + */ + void reset() throws InterruptedIOException; + } +} diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index bb6cbb5..ed7202a 100644 --- hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -33,12 +33,10 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.BrokenBarrierException; -import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; @@ -59,15 +57,8 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.AsyncProcess.ListRowAccess; -import org.apache.hadoop.hbase.client.AsyncProcess.TaskCountChecker; -import org.apache.hadoop.hbase.client.AsyncProcess.RowChecker.ReturnCode; -import org.apache.hadoop.hbase.client.AsyncProcess.RowCheckerHost; -import org.apache.hadoop.hbase.client.AsyncProcess.RequestSizeChecker; +import org.apache.hadoop.hbase.client.AsyncProcessTask.ListRowAccess; import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; @@ -78,60 +69,64 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestRule; import org.mockito.Mockito; -import org.apache.hadoop.hbase.client.AsyncProcess.RowChecker; -import org.apache.hadoop.hbase.client.AsyncProcess.SubmittedSizeChecker; +import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ServerStatistics; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.testclassification.ClientTests; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; + @Category({ClientTests.class, MediumTests.class}) public class TestAsyncProcess { @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()). withLookingForStuckThread(true).build(); - private final static Log LOG = LogFactory.getLog(TestAsyncProcess.class); + private static final Log LOG = LogFactory.getLog(TestAsyncProcess.class); private static final TableName DUMMY_TABLE = TableName.valueOf("DUMMY_TABLE"); private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes(); private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes(); private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes(); private static final byte[] FAILS = "FAILS".getBytes(); - private static final Configuration conf = new Configuration(); - - private static ServerName sn = ServerName.valueOf("s1:1,1"); - private static ServerName sn2 = ServerName.valueOf("s2:2,2"); - private static ServerName sn3 = ServerName.valueOf("s3:3,3"); - private static HRegionInfo hri1 = + private static final Configuration CONF = new Configuration(); + private static final ConnectionConfiguration CONNECTION_CONFIG = new ConnectionConfiguration(CONF); + private static final ServerName sn = ServerName.valueOf("s1:1,1"); + private static final ServerName sn2 = ServerName.valueOf("s2:2,2"); + private static final ServerName sn3 = ServerName.valueOf("s3:3,3"); + private static final HRegionInfo hri1 = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1); - private static HRegionInfo hri2 = + private static final HRegionInfo hri2 = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_2, HConstants.EMPTY_END_ROW, false, 2); - private static HRegionInfo hri3 = + private static final HRegionInfo hri3 = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_3, HConstants.EMPTY_END_ROW, false, 3); - private static HRegionLocation loc1 = new HRegionLocation(hri1, sn); - private static HRegionLocation loc2 = new HRegionLocation(hri2, sn); - private static HRegionLocation loc3 = new HRegionLocation(hri3, sn2); + private static final HRegionLocation loc1 = new HRegionLocation(hri1, sn); + private static final HRegionLocation loc2 = new HRegionLocation(hri2, sn); + private static final HRegionLocation loc3 = new HRegionLocation(hri3, sn2); // Replica stuff - private static HRegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1), + private static final HRegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1), hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2); - private static HRegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1); - private static RegionLocations hrls1 = new RegionLocations(new HRegionLocation(hri1, sn), + private static final HRegionInfo hri2r1 = RegionReplicaUtil.getRegionInfoForReplica(hri2, 1); + private static final RegionLocations hrls1 = new RegionLocations(new HRegionLocation(hri1, sn), new HRegionLocation(hri1r1, sn2), new HRegionLocation(hri1r2, sn3)); - private static RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2), + private static final RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2), new HRegionLocation(hri2r1, sn3)); - private static RegionLocations hrls3 = new RegionLocations(new HRegionLocation(hri3, sn3), null); + private static final RegionLocations hrls3 = new RegionLocations(new HRegionLocation(hri3, sn3), null); private static final String success = "success"; private static Exception failure = new Exception("failure"); - private static int NB_RETRIES = 3; + private static final int NB_RETRIES = 3; + private static final int RPC_TIMEOUT = CONF.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT); + private static final int OPERATION_TIMEOUT = CONF.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, + HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); @BeforeClass public static void beforeClass(){ - conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, NB_RETRIES); + CONF.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, NB_RETRIES); } static class CountingThreadFactory implements ThreadFactory { @@ -153,20 +148,21 @@ public class TestAsyncProcess { final AtomicInteger nbActions = new AtomicInteger(); public List allReqs = new ArrayList(); public AtomicInteger callsCt = new AtomicInteger(); - private static int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - private static int operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); + private long previousTimeout = -1; + final ExecutorService service; @Override - protected AsyncRequestFutureImpl createAsyncRequestFuture(TableName tableName, - List actions, long nonceGroup, ExecutorService pool, - Batch.Callback callback, Object[] results, boolean needResults, - CancellableRegionServerCallable callable, int curTimeout) { + protected AsyncRequestFutureImpl createAsyncRequestFuture( + AsyncProcessTask task, List actions, long nonceGroup) { // Test HTable has tableName of null, so pass DUMMY_TABLE + AsyncProcessTask wrap = new AsyncProcessTask(task){ + @Override + public TableName getTableName() { + return DUMMY_TABLE; + } + }; AsyncRequestFutureImpl r = new MyAsyncRequestFutureImpl( - DUMMY_TABLE, actions, nonceGroup, getPool(pool), needResults, - results, callback, callable, operationTimeout, rpcTimeout, this); + wrap, actions, nonceGroup, this); allReqs.add(r); return r; } @@ -176,49 +172,54 @@ public class TestAsyncProcess { } public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) { - super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, - new SynchronousQueue(), new CountingThreadFactory(nbThreads)), - new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf), rpcTimeout, - operationTimeout); + super(hc, conf, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf)); + service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, + new SynchronousQueue<>(), new CountingThreadFactory(nbThreads)); } public MyAsyncProcess( ClusterConnection hc, Configuration conf, boolean useGlobalErrors) { - super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, - new SynchronousQueue(), new CountingThreadFactory(new AtomicInteger())), - new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf), - rpcTimeout, operationTimeout); + super(hc, conf, + new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf)); + service = Executors.newFixedThreadPool(5); } - public MyAsyncProcess(ClusterConnection hc, Configuration conf, boolean useGlobalErrors, - @SuppressWarnings("unused") boolean dummy) { - super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, - new SynchronousQueue(), new CountingThreadFactory(new AtomicInteger())) { - @Override - public void execute(Runnable command) { - throw new RejectedExecutionException("test under failure"); - } - }, - new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf), - rpcTimeout, operationTimeout); + public AsyncRequestFuture submit(ExecutorService pool, TableName tableName, + List rows, boolean atLeastOne, Batch.Callback callback, + boolean needResults) throws InterruptedIOException { + AsyncProcessTask task = AsyncProcessTask.newBuilder(callback) + .setPool(pool == null ? service : pool) + .setTableName(tableName) + .setRowAccess(rows) + .setSubmittedRows(atLeastOne ? SubmittedRows.AT_LEAST_ONE : SubmittedRows.NORMAL) + .setNeedResults(needResults) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .build(); + return submit(task); + } + + public AsyncRequestFuture submit(TableName tableName, + final List rows, boolean atLeastOne, Batch.Callback callback, + boolean needResults) throws InterruptedIOException { + return submit(null, tableName, rows, atLeastOne, callback, needResults); } @Override - public AsyncRequestFuture submit(TableName tableName, RowAccess rows, - boolean atLeastOne, Callback callback, boolean needResults) + public AsyncRequestFuture submit(AsyncProcessTask task) throws InterruptedIOException { + previousTimeout = task.getRpcTimeout(); // We use results in tests to check things, so override to always save them. - return super.submit(DUMMY_TABLE, rows, atLeastOne, callback, true); + AsyncProcessTask wrap = new AsyncProcessTask(task) { + @Override + public boolean getNeedResults() { + return true; + } + }; + return super.submit(wrap); } @Override - public AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName, - List rows, Batch.Callback callback, Object[] results, - CancellableRegionServerCallable callable, int curTimeout) { - previousTimeout = curTimeout; - return super.submitAll(pool, tableName, rows, callback, results, callable, curTimeout); - } - @Override protected RpcRetryingCaller createCaller( CancellableRegionServerCallable callable, int rpcTimeout) { callsCt.incrementAndGet(); @@ -260,12 +261,9 @@ public class TestAsyncProcess { static class MyAsyncRequestFutureImpl extends AsyncRequestFutureImpl { - public MyAsyncRequestFutureImpl(TableName tableName, List actions, long nonceGroup, - ExecutorService pool, boolean needResults, Object[] results, - Batch.Callback callback, CancellableRegionServerCallable callable, int operationTimeout, - int rpcTimeout, AsyncProcess asyncProcess) { - super(tableName, actions, nonceGroup, pool, needResults, - results, callback, callable, operationTimeout, rpcTimeout, asyncProcess); + public MyAsyncRequestFutureImpl(AsyncProcessTask task, List actions, + long nonceGroup, AsyncProcess asyncProcess) { + super(task, actions, nonceGroup, asyncProcess); } @Override @@ -483,7 +481,7 @@ public class TestAsyncProcess { final boolean usedRegions[]; protected MyConnectionImpl2(List hrl) throws IOException { - super(conf); + super(CONF); this.hrl = hrl; this.usedRegions = new boolean[hrl.size()]; } @@ -553,19 +551,7 @@ public class TestAsyncProcess { long putsHeapSize = writeBuffer; doSubmitRequest(writeBuffer, putsHeapSize); } - @Test - public void testIllegalArgument() throws IOException { - ClusterConnection conn = createHConnection(); - final long maxHeapSizePerRequest = conn.getConfiguration().getLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, - AsyncProcess.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); - conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, -1); - try { - MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true); - fail("The maxHeapSizePerRequest must be bigger than zero"); - } catch (IllegalArgumentException e) { - } - conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, maxHeapSizePerRequest); - } + @Test public void testSubmitLargeRequestWithUnlimitedSize() throws Exception { long maxHeapSizePerRequest = Long.MAX_VALUE; @@ -601,10 +587,13 @@ public class TestAsyncProcess { private void doSubmitRequest(long maxHeapSizePerRequest, long putsHeapSize) throws Exception { ClusterConnection conn = createHConnection(); - final long defaultHeapSizePerRequest = conn.getConfiguration().getLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, - AsyncProcess.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); - conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, maxHeapSizePerRequest); - BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE); + final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); + final long defaultHeapSizePerRequest = conn.getConfiguration().getLong( + SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, + SimpleRequestController.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE); + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class.getName()); + conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, maxHeapSizePerRequest); // sn has two regions long putSizeSN = 0; @@ -630,11 +619,12 @@ public class TestAsyncProcess { + ", maxHeapSizePerRequest:" + maxHeapSizePerRequest + ", minCountSnRequest:" + minCountSnRequest + ", minCountSn2Request:" + minCountSn2Request); - try (HTable ht = new HTable(conn, bufferParam)) { - MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true); - ht.mutator.ap = ap; - Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get()); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + try (HTable ht = new HTable(conn, mutator)) { + Assert.assertEquals(0L, ht.mutator.getCurrentWriteBufferSize()); ht.put(puts); List reqs = ap.allReqs; @@ -680,12 +670,17 @@ public class TestAsyncProcess { assertEquals(putSizeSN2, (long) sizePerServers.get(sn2)); } // restore config. - conn.getConfiguration().setLong(AsyncProcess.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, defaultHeapSizePerRequest); + conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, defaultHeapSizePerRequest); + if (defaultClazz != null) { + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + defaultClazz); + } } + @Test public void testSubmit() throws Exception { ClusterConnection hc = createHConnection(); - AsyncProcess ap = new MyAsyncProcess(hc, conf); + MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); List puts = new ArrayList(); puts.add(createPut(1, true)); @@ -704,7 +699,7 @@ public class TestAsyncProcess { updateCalled.incrementAndGet(); } }; - AsyncProcess ap = new MyAsyncProcess(hc, conf); + MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); List puts = new ArrayList(); puts.add(createPut(1, true)); @@ -717,13 +712,16 @@ public class TestAsyncProcess { @Test public void testSubmitBusyRegion() throws Exception { - ClusterConnection hc = createHConnection(); - AsyncProcess ap = new MyAsyncProcess(hc, conf); - + ClusterConnection conn = createHConnection(); + final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class.getName()); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); + SimpleRequestController controller = (SimpleRequestController) ap.requestController; List puts = new ArrayList(); puts.add(createPut(1, true)); - for (int i = 0; i != ap.maxConcurrentTasksPerRegion; ++i) { + for (int i = 0; i != controller.maxConcurrentTasksPerRegion; ++i) { ap.incTaskCounters(Arrays.asList(hri1.getRegionName()), sn); } ap.submit(null, DUMMY_TABLE, puts, false, null, false); @@ -732,15 +730,22 @@ public class TestAsyncProcess { ap.decTaskCounters(Arrays.asList(hri1.getRegionName()), sn); ap.submit(null, DUMMY_TABLE, puts, false, null, false); Assert.assertEquals(0, puts.size()); + if (defaultClazz != null) { + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + defaultClazz); + } } @Test public void testSubmitBusyRegionServer() throws Exception { - ClusterConnection hc = createHConnection(); - AsyncProcess ap = new MyAsyncProcess(hc, conf); - - ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer)); + ClusterConnection conn = createHConnection(); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); + final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class.getName()); + SimpleRequestController controller = (SimpleRequestController) ap.requestController; + controller.taskCounterPerServer.put(sn2, new AtomicInteger(controller.maxConcurrentTasksPerServer)); List puts = new ArrayList(); puts.add(createPut(1, true)); @@ -751,14 +756,18 @@ public class TestAsyncProcess { ap.submit(null, DUMMY_TABLE, puts, false, null, false); Assert.assertEquals(" puts=" + puts, 1, puts.size()); - ap.taskCounterPerServer.put(sn2, new AtomicInteger(ap.maxConcurrentTasksPerServer - 1)); + controller.taskCounterPerServer.put(sn2, new AtomicInteger(controller.maxConcurrentTasksPerServer - 1)); ap.submit(null, DUMMY_TABLE, puts, false, null, false); Assert.assertTrue(puts.isEmpty()); + if (defaultClazz != null) { + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + defaultClazz); + } } @Test public void testFail() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false); + MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false); List puts = new ArrayList(); Put p = createPut(1, false); @@ -784,10 +793,15 @@ public class TestAsyncProcess { @Test public void testSubmitTrue() throws IOException { - final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false); - ap.tasksInProgress.incrementAndGet(); - final AtomicInteger ai = new AtomicInteger(ap.maxConcurrentTasksPerRegion); - ap.taskCounterPerRegion.put(hri1.getRegionName(), ai); + ClusterConnection conn = createHConnection(); + final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, false); + final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class.getName()); + SimpleRequestController controller = (SimpleRequestController) ap.requestController; + controller.tasksInProgress.incrementAndGet(); + final AtomicInteger ai = new AtomicInteger(controller.maxConcurrentTasksPerRegion); + controller.taskCounterPerRegion.put(hri1.getRegionName(), ai); final AtomicBoolean checkPoint = new AtomicBoolean(false); final AtomicBoolean checkPoint2 = new AtomicBoolean(false); @@ -798,7 +812,7 @@ public class TestAsyncProcess { Threads.sleep(1000); Assert.assertFalse(checkPoint.get()); // TODO: this is timing-dependent ai.decrementAndGet(); - ap.tasksInProgress.decrementAndGet(); + controller.tasksInProgress.decrementAndGet(); checkPoint2.set(true); } }; @@ -819,11 +833,15 @@ public class TestAsyncProcess { while (!checkPoint2.get()){ Threads.sleep(1); } + if (defaultClazz != null) { + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + defaultClazz); + } } @Test public void testFailAndSuccess() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false); + MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false); List puts = new ArrayList(); puts.add(createPut(1, false)); @@ -850,7 +868,7 @@ public class TestAsyncProcess { @Test public void testFlush() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false); + MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false); List puts = new ArrayList(); puts.add(createPut(1, false)); @@ -868,24 +886,32 @@ public class TestAsyncProcess { @Test public void testTaskCountWithoutClientBackoffPolicy() throws IOException, InterruptedException { ClusterConnection hc = createHConnection(); - MyAsyncProcess ap = new MyAsyncProcess(hc, conf, false); + MyAsyncProcess ap = new MyAsyncProcess(hc, CONF, false); testTaskCount(ap); } @Test public void testTaskCountWithClientBackoffPolicy() throws IOException, InterruptedException { - Configuration copyConf = new Configuration(conf); + Configuration copyConf = new Configuration(CONF); copyConf.setBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, true); MyClientBackoffPolicy bp = new MyClientBackoffPolicy(); - ClusterConnection hc = createHConnection(); - Mockito.when(hc.getConfiguration()).thenReturn(copyConf); - Mockito.when(hc.getStatisticsTracker()).thenReturn(ServerStatisticTracker.create(copyConf)); - Mockito.when(hc.getBackoffPolicy()).thenReturn(bp); - MyAsyncProcess ap = new MyAsyncProcess(hc, copyConf, false); + ClusterConnection conn = createHConnection(); + Mockito.when(conn.getConfiguration()).thenReturn(copyConf); + Mockito.when(conn.getStatisticsTracker()).thenReturn(ServerStatisticTracker.create(copyConf)); + Mockito.when(conn.getBackoffPolicy()).thenReturn(bp); + final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class.getName()); + MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, false); testTaskCount(ap); + if (defaultClazz != null) { + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + defaultClazz); + } } - private void testTaskCount(AsyncProcess ap) throws InterruptedIOException, InterruptedException { + private void testTaskCount(MyAsyncProcess ap) throws InterruptedIOException, InterruptedException { + SimpleRequestController controller = (SimpleRequestController) ap.requestController; List puts = new ArrayList<>(); for (int i = 0; i != 3; ++i) { puts.add(createPut(1, true)); @@ -896,18 +922,24 @@ public class TestAsyncProcess { ap.waitForMaximumCurrentTasks(0, null); // More time to wait if there are incorrect task count. TimeUnit.SECONDS.sleep(1); - assertEquals(0, ap.tasksInProgress.get()); - for (AtomicInteger count : ap.taskCounterPerRegion.values()) { + assertEquals(0, controller.tasksInProgress.get()); + for (AtomicInteger count : controller.taskCounterPerRegion.values()) { assertEquals(0, count.get()); } - for (AtomicInteger count : ap.taskCounterPerServer.values()) { + for (AtomicInteger count : controller.taskCounterPerServer.values()) { assertEquals(0, count.get()); } } @Test public void testMaxTask() throws Exception { - final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, false); + ClusterConnection conn = createHConnection(); + final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY); + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + SimpleRequestController.class.getName()); + final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, false); + SimpleRequestController controller = (SimpleRequestController) ap.requestController; + for (int i = 0; i < 1000; i++) { ap.incTaskCounters(Arrays.asList("dummy".getBytes()), sn); @@ -940,7 +972,7 @@ public class TestAsyncProcess { @Override public void run() { Threads.sleep(sleepTime); - while (ap.tasksInProgress.get() > 0) { + while (controller.tasksInProgress.get() > 0) { ap.decTaskCounters(Arrays.asList("dummy".getBytes()), sn); } } @@ -953,6 +985,10 @@ public class TestAsyncProcess { //Adds 100 to secure us against approximate timing. Assert.assertTrue(start + 100L + sleepTime > end); + if (defaultClazz != null) { + conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, + defaultClazz); + } } private static ClusterConnection createHConnection() throws IOException { @@ -999,38 +1035,53 @@ public class TestAsyncProcess { NonceGenerator ng = Mockito.mock(NonceGenerator.class); Mockito.when(ng.getNonceGroup()).thenReturn(HConstants.NO_NONCE); Mockito.when(hc.getNonceGenerator()).thenReturn(ng); - Mockito.when(hc.getConfiguration()).thenReturn(conf); + Mockito.when(hc.getConfiguration()).thenReturn(CONF); + Mockito.when(hc.getConnectionConfiguration()).thenReturn(CONNECTION_CONFIG); return hc; } @Test public void testHTablePutSuccess() throws Exception { - BufferedMutatorImpl ht = Mockito.mock(BufferedMutatorImpl.class); - ht.ap = new MyAsyncProcess(createHConnection(), conf, true); + ClusterConnection conn = createHConnection(); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap); Put put = createPut(1, true); - Assert.assertEquals(0, ht.getWriteBufferSize()); + Assert.assertEquals(conn.getConnectionConfiguration().getWriteBufferSize(), ht.getWriteBufferSize()); + Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); ht.mutate(put); - Assert.assertEquals(0, ht.getWriteBufferSize()); + ht.flush(); + Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); + } + + @Test + public void testBufferedMutatorImplWithSharedPool() throws Exception { + ClusterConnection conn = createHConnection(); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutator ht = new BufferedMutatorImpl(conn, bufferParam, ap); + + ht.close(); + assertFalse(ap.service.isShutdown()); } private void doHTableFailedPut(boolean bufferOn) throws Exception { ClusterConnection conn = createHConnection(); - BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); if (bufferOn) { bufferParam.writeBufferSize(1024L * 1024L); } else { bufferParam.writeBufferSize(0L); } - - HTable ht = new HTable(conn, bufferParam); - MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true); - ht.mutator.ap = ap; + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + HTable ht = new HTable(conn, mutator); Put put = createPut(1, false); - Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get()); + Assert.assertEquals(0L, ht.mutator.getCurrentWriteBufferSize()); try { ht.put(put); if (bufferOn) { @@ -1039,7 +1090,7 @@ public class TestAsyncProcess { Assert.fail(); } catch (RetriesExhaustedException expected) { } - Assert.assertEquals(0L, ht.mutator.currentWriteBufferSize.get()); + Assert.assertEquals(0L, ht.mutator.getCurrentWriteBufferSize()); // The table should have sent one request, maybe after multiple attempts AsyncRequestFuture ars = null; for (AsyncRequestFuture someReqs : ap.allReqs) { @@ -1067,10 +1118,10 @@ public class TestAsyncProcess { @Test public void testHTableFailedPutAndNewPut() throws Exception { ClusterConnection conn = createHConnection(); - BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null, - new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(0)); - MyAsyncProcess ap = new MyAsyncProcess(conn, conf, true); - mutator.ap = ap; + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE) + .writeBufferSize(0); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); Put p = createPut(1, false); mutator.mutate(p); @@ -1083,202 +1134,13 @@ public class TestAsyncProcess { // puts, we may raise an exception in the middle of the list. It's then up to the caller to // manage what was inserted, what was tried but failed, and what was not even tried. p = createPut(1, true); - Assert.assertEquals(0, mutator.writeAsyncBuffer.size()); + Assert.assertEquals(0, mutator.size()); try { mutator.mutate(p); Assert.fail(); } catch (RetriesExhaustedException expected) { } - Assert.assertEquals("the put should not been inserted.", 0, mutator.writeAsyncBuffer.size()); - } - - @Test - public void testTaskCheckerHost() throws IOException { - final int maxTotalConcurrentTasks = 100; - final int maxConcurrentTasksPerServer = 2; - final int maxConcurrentTasksPerRegion = 1; - final AtomicLong tasksInProgress = new AtomicLong(0); - final Map taskCounterPerServer = new HashMap<>(); - final Map taskCounterPerRegion = new HashMap<>(); - TaskCountChecker countChecker = new TaskCountChecker( - maxTotalConcurrentTasks, - maxConcurrentTasksPerServer, - maxConcurrentTasksPerRegion, - tasksInProgress, taskCounterPerServer, taskCounterPerRegion); - final long maxHeapSizePerRequest = 2 * 1024 * 1024; - // unlimiited - RequestSizeChecker sizeChecker = new RequestSizeChecker(maxHeapSizePerRequest); - RowCheckerHost checkerHost = new RowCheckerHost(Arrays.asList(countChecker, sizeChecker)); - - ReturnCode loc1Code = checkerHost.canTakeOperation(loc1, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, loc1Code); - - ReturnCode loc1Code_2 = checkerHost.canTakeOperation(loc1, maxHeapSizePerRequest); - // rejected for size - assertNotEquals(RowChecker.ReturnCode.INCLUDE, loc1Code_2); - - ReturnCode loc2Code = checkerHost.canTakeOperation(loc2, maxHeapSizePerRequest); - // rejected for size - assertNotEquals(RowChecker.ReturnCode.INCLUDE, loc2Code); - - // fill the task slots for loc3. - taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(100)); - taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(100)); - - ReturnCode loc3Code = checkerHost.canTakeOperation(loc3, 1L); - // rejected for count - assertNotEquals(RowChecker.ReturnCode.INCLUDE, loc3Code); - - // release the task slots for loc3. - taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(0)); - taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(0)); - - ReturnCode loc3Code_2 = checkerHost.canTakeOperation(loc3, 1L); - assertEquals(RowChecker.ReturnCode.INCLUDE, loc3Code_2); - } - - @Test - public void testRequestSizeCheckerr() throws IOException { - final long maxHeapSizePerRequest = 2 * 1024 * 1024; - final ClusterConnection conn = createHConnection(); - RequestSizeChecker checker = new RequestSizeChecker(maxHeapSizePerRequest); - - // inner state is unchanged. - for (int i = 0; i != 10; ++i) { - ReturnCode code = checker.canTakeOperation(loc1, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - code = checker.canTakeOperation(loc2, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - } - - // accept the data located on loc1 region. - ReturnCode acceptCode = checker.canTakeOperation(loc1, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, acceptCode); - checker.notifyFinal(acceptCode, loc1, maxHeapSizePerRequest); - - // the sn server reachs the limit. - for (int i = 0; i != 10; ++i) { - ReturnCode code = checker.canTakeOperation(loc1, maxHeapSizePerRequest); - assertNotEquals(RowChecker.ReturnCode.INCLUDE, code); - code = checker.canTakeOperation(loc2, maxHeapSizePerRequest); - assertNotEquals(RowChecker.ReturnCode.INCLUDE, code); - } - - // the request to sn2 server should be accepted. - for (int i = 0; i != 10; ++i) { - ReturnCode code = checker.canTakeOperation(loc3, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - } - - checker.reset(); - for (int i = 0; i != 10; ++i) { - ReturnCode code = checker.canTakeOperation(loc1, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - code = checker.canTakeOperation(loc2, maxHeapSizePerRequest); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - } - } - - @Test - public void testSubmittedSizeChecker() { - final long maxHeapSizeSubmit = 2 * 1024 * 1024; - SubmittedSizeChecker checker = new SubmittedSizeChecker(maxHeapSizeSubmit); - - for (int i = 0; i != 10; ++i) { - ReturnCode include = checker.canTakeOperation(loc1, 100000); - assertEquals(ReturnCode.INCLUDE, include); - } - - for (int i = 0; i != 10; ++i) { - checker.notifyFinal(ReturnCode.INCLUDE, loc1, maxHeapSizeSubmit); - } - - for (int i = 0; i != 10; ++i) { - ReturnCode include = checker.canTakeOperation(loc1, 100000); - assertEquals(ReturnCode.END, include); - } - for (int i = 0; i != 10; ++i) { - ReturnCode include = checker.canTakeOperation(loc2, 100000); - assertEquals(ReturnCode.END, include); - } - checker.reset(); - for (int i = 0; i != 10; ++i) { - ReturnCode include = checker.canTakeOperation(loc1, 100000); - assertEquals(ReturnCode.INCLUDE, include); - } - } - @Test - public void testTaskCountChecker() throws InterruptedIOException { - long rowSize = 12345; - int maxTotalConcurrentTasks = 100; - int maxConcurrentTasksPerServer = 2; - int maxConcurrentTasksPerRegion = 1; - AtomicLong tasksInProgress = new AtomicLong(0); - Map taskCounterPerServer = new HashMap<>(); - Map taskCounterPerRegion = new HashMap<>(); - TaskCountChecker checker = new TaskCountChecker( - maxTotalConcurrentTasks, - maxConcurrentTasksPerServer, - maxConcurrentTasksPerRegion, - tasksInProgress, taskCounterPerServer, taskCounterPerRegion); - - // inner state is unchanged. - for (int i = 0; i != 10; ++i) { - ReturnCode code = checker.canTakeOperation(loc1, rowSize); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - } - // add loc1 region. - ReturnCode code = checker.canTakeOperation(loc1, rowSize); - assertEquals(RowChecker.ReturnCode.INCLUDE, code); - checker.notifyFinal(code, loc1, rowSize); - - // fill the task slots for loc1. - taskCounterPerRegion.put(loc1.getRegionInfo().getRegionName(), new AtomicInteger(100)); - taskCounterPerServer.put(loc1.getServerName(), new AtomicInteger(100)); - - // the region was previously accepted, so it must be accpted now. - for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { - ReturnCode includeCode = checker.canTakeOperation(loc1, rowSize); - assertEquals(RowChecker.ReturnCode.INCLUDE, includeCode); - checker.notifyFinal(includeCode, loc1, rowSize); - } - - // fill the task slots for loc3. - taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(100)); - taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(100)); - - // no task slots. - for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { - ReturnCode excludeCode = checker.canTakeOperation(loc3, rowSize); - assertNotEquals(RowChecker.ReturnCode.INCLUDE, excludeCode); - checker.notifyFinal(excludeCode, loc3, rowSize); - } - - // release the tasks for loc3. - taskCounterPerRegion.put(loc3.getRegionInfo().getRegionName(), new AtomicInteger(0)); - taskCounterPerServer.put(loc3.getServerName(), new AtomicInteger(0)); - - // add loc3 region. - ReturnCode code3 = checker.canTakeOperation(loc3, rowSize); - assertEquals(RowChecker.ReturnCode.INCLUDE, code3); - checker.notifyFinal(code3, loc3, rowSize); - - // the region was previously accepted, so it must be accpted now. - for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { - ReturnCode includeCode = checker.canTakeOperation(loc3, rowSize); - assertEquals(RowChecker.ReturnCode.INCLUDE, includeCode); - checker.notifyFinal(includeCode, loc3, rowSize); - } - - checker.reset(); - // the region was previously accepted, - // but checker have reseted and task slots for loc1 is full. - // So it must be rejected now. - for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { - ReturnCode includeCode = checker.canTakeOperation(loc1, rowSize); - assertNotEquals(RowChecker.ReturnCode.INCLUDE, includeCode); - checker.notifyFinal(includeCode, loc1, rowSize); - } + Assert.assertEquals("the put should not been inserted.", 0, mutator.size()); } @Test @@ -1302,9 +1164,12 @@ public class TestAsyncProcess { @Test public void testBatch() throws IOException, InterruptedException { - ClusterConnection conn = new MyConnectionImpl(conf); - HTable ht = new HTable(conn, new BufferedMutatorParams(DUMMY_TABLE)); - ht.multiAp = new MyAsyncProcess(conn, conf, false); + ClusterConnection conn = new MyConnectionImpl(CONF); + MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + HTable ht = new HTable(conn, mutator); + ht.multiAp = new MyAsyncProcess(conn, CONF, false); List puts = new ArrayList(); puts.add(createPut(1, true)); @@ -1332,18 +1197,16 @@ public class TestAsyncProcess { } @Test public void testErrorsServers() throws IOException { - Configuration configuration = new Configuration(conf); + Configuration configuration = new Configuration(CONF); ClusterConnection conn = new MyConnectionImpl(configuration); - BufferedMutatorImpl mutator = - new BufferedMutatorImpl(conn, null, null, new BufferedMutatorParams(DUMMY_TABLE)); - configuration.setBoolean(ConnectionImplementation.RETRIES_BY_SERVER_KEY, true); - MyAsyncProcess ap = new MyAsyncProcess(conn, configuration, true); - mutator.ap = ap; + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + configuration.setBoolean(ConnectionImplementation.RETRIES_BY_SERVER_KEY, true); - Assert.assertNotNull(mutator.ap.createServerErrorTracker()); - Assert.assertTrue(mutator.ap.serverTrackerTimeout > 200); - mutator.ap.serverTrackerTimeout = 1; + Assert.assertNotNull(ap.createServerErrorTracker()); + Assert.assertTrue(ap.serverTrackerTimeout > 200); + ap.serverTrackerTimeout = 1; Put p = createPut(1, false); mutator.mutate(p); @@ -1361,14 +1224,15 @@ public class TestAsyncProcess { public void testReadAndWriteTimeout() throws IOException { final long readTimeout = 10 * 1000; final long writeTimeout = 20 * 1000; - Configuration copyConf = new Configuration(conf); + Configuration copyConf = new Configuration(CONF); copyConf.setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, readTimeout); copyConf.setLong(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, writeTimeout); ClusterConnection conn = createHConnection(); Mockito.when(conn.getConfiguration()).thenReturn(copyConf); - BufferedMutatorParams bufferParam = new BufferedMutatorParams(DUMMY_TABLE); - try (HTable ht = new HTable(conn, bufferParam)) { - MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, true); + MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, true); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + try (HTable ht = new HTable(conn, mutator)) { ht.multiAp = ap; List gets = new LinkedList<>(); gets.add(new Get(DUMMY_BYTES_1)); @@ -1399,12 +1263,12 @@ public class TestAsyncProcess { @Test public void testGlobalErrors() throws IOException { - ClusterConnection conn = new MyConnectionImpl(conf); - BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE); - AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, conf, new IOException("test")); - mutator.ap = ap; + ClusterConnection conn = new MyConnectionImpl(CONF); + AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, CONF, new IOException("test")); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - Assert.assertNotNull(mutator.ap.createServerErrorTracker()); + Assert.assertNotNull(ap.createServerErrorTracker()); Put p = createPut(1, true); mutator.mutate(p); @@ -1421,13 +1285,11 @@ public class TestAsyncProcess { @Test public void testCallQueueTooLarge() throws IOException { - ClusterConnection conn = new MyConnectionImpl(conf); - BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE); - AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, conf, new CallQueueTooBigException()); - mutator.ap = ap; - - Assert.assertNotNull(mutator.ap.createServerErrorTracker()); - + ClusterConnection conn = new MyConnectionImpl(CONF); + AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, CONF, new CallQueueTooBigException()); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + Assert.assertNotNull(ap.createServerErrorTracker()); Put p = createPut(1, true); mutator.mutate(p); @@ -1459,10 +1321,11 @@ public class TestAsyncProcess { } MyConnectionImpl2 con = new MyConnectionImpl2(hrls); - HTable ht = new HTable(con, new BufferedMutatorParams(DUMMY_TABLE)); - MyAsyncProcess ap = new MyAsyncProcess(con, conf, con.nbThreads); + MyAsyncProcess ap = new MyAsyncProcess(con, CONF, con.nbThreads); + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(con , bufferParam, ap); + HTable ht = new HTable(con, mutator); ht.multiAp = ap; - ht.batch(gets, null); Assert.assertEquals(ap.nbActions.get(), NB_REGS); @@ -1482,7 +1345,16 @@ public class TestAsyncProcess { // One region has no replica, so the main call succeeds for it. MyAsyncProcessWithReplicas ap = createReplicaAp(10, 1000, 0); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3); - AsyncRequestFuture ars = ap.submitAll(null,DUMMY_TABLE, rows, null, new Object[3]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[3]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.TRUE, RR.TRUE, RR.FALSE); Assert.assertEquals(2, ap.getReplicaCallCount()); } @@ -1492,7 +1364,16 @@ public class TestAsyncProcess { // Main call succeeds before replica calls are kicked off. MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 10, 0); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2, DUMMY_BYTES_3); - AsyncRequestFuture ars = ap.submitAll(null, DUMMY_TABLE, rows, null, new Object[3]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[3]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.FALSE, RR.FALSE, RR.FALSE); Assert.assertEquals(0, ap.getReplicaCallCount()); } @@ -1502,7 +1383,16 @@ public class TestAsyncProcess { // Either main or replica can succeed. MyAsyncProcessWithReplicas ap = createReplicaAp(0, 0, 0); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); - AsyncRequestFuture ars = ap.submitAll(null, DUMMY_TABLE, rows, null, new Object[2]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[2]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.DONT_CARE, RR.DONT_CARE); long replicaCalls = ap.getReplicaCallCount(); Assert.assertTrue(replicaCalls >= 0); @@ -1517,7 +1407,16 @@ public class TestAsyncProcess { MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0); ap.setPrimaryCallDelay(sn2, 2000); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); - AsyncRequestFuture ars = ap.submitAll(null ,DUMMY_TABLE, rows, null, new Object[2]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[2]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.FALSE, RR.TRUE); Assert.assertEquals(1, ap.getReplicaCallCount()); } @@ -1530,7 +1429,16 @@ public class TestAsyncProcess { MyAsyncProcessWithReplicas ap = createReplicaAp(1000, 0, 0, 0); ap.addFailures(hri1, hri2); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); - AsyncRequestFuture ars = ap.submitAll(null, DUMMY_TABLE, rows, null, new Object[2]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[2]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.FAILED, RR.FAILED); Assert.assertEquals(0, ap.getReplicaCallCount()); } @@ -1542,7 +1450,16 @@ public class TestAsyncProcess { MyAsyncProcessWithReplicas ap = createReplicaAp(0, 1000, 1000, 0); ap.addFailures(hri1, hri1r2, hri2); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); - AsyncRequestFuture ars = ap.submitAll(null, DUMMY_TABLE, rows, null, new Object[2]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[2]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.TRUE, RR.TRUE); Assert.assertEquals(2, ap.getReplicaCallCount()); } @@ -1554,7 +1471,16 @@ public class TestAsyncProcess { MyAsyncProcessWithReplicas ap = createReplicaAp(500, 1000, 0, 0); ap.addFailures(hri1, hri1r1, hri1r2, hri2r1); List rows = makeTimelineGets(DUMMY_BYTES_1, DUMMY_BYTES_2); - AsyncRequestFuture ars = ap.submitAll(null, DUMMY_TABLE, rows, null, new Object[2]); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(ap.service) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(rows) + .setResults(new Object[2]) + .setSubmittedRows(SubmittedRows.ALL) + .build(); + AsyncRequestFuture ars = ap.submit(task); verifyReplicaResult(ars, RR.FAILED, RR.FALSE); // We should get 3 exceptions, for main + 2 replicas for DUMMY_BYTES_1 Assert.assertEquals(3, ars.getErrors().getNumExceptions()); @@ -1583,6 +1509,13 @@ public class TestAsyncProcess { return ap; } + private static BufferedMutatorParams createBufferedMutatorParams(MyAsyncProcess ap, TableName name) { + return new BufferedMutatorParams(name) + .pool(ap.service) + .rpcTimeout(RPC_TIMEOUT) + .opertationTimeout(OPERATION_TIMEOUT); + } + private static List makeTimelineGets(byte[]... rows) { List result = new ArrayList(); for (byte[] row : rows) { @@ -1663,14 +1596,9 @@ public class TestAsyncProcess { } static class AsyncProcessForThrowableCheck extends AsyncProcess { - private static int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - private static int operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf, - ExecutorService pool) { - super(hc, conf, pool, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory( - conf), rpcTimeout, operationTimeout); + public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf) { + super(hc, conf, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory( + conf)); } } @@ -1681,56 +1609,22 @@ public class TestAsyncProcess { MyThreadPoolExecutor myPool = new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, new LinkedBlockingQueue(200)); - AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, conf, myPool); + AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, CONF); List puts = new ArrayList(); puts.add(createPut(1, true)); - - ap.submit(null, DUMMY_TABLE, puts, false, null, false); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(myPool) + .setRpcTimeout(RPC_TIMEOUT) + .setOperationTimeout(OPERATION_TIMEOUT) + .setTableName(DUMMY_TABLE) + .setRowAccess(puts) + .setSubmittedRows(SubmittedRows.NORMAL) + .build(); + ap.submit(task); Assert.assertTrue(puts.isEmpty()); } - @Test - public void testWaitForMaximumCurrentTasks() throws Exception { - final AtomicLong tasks = new AtomicLong(0); - final AtomicInteger max = new AtomicInteger(0); - final CyclicBarrier barrier = new CyclicBarrier(2); - final AsyncProcess ap = new MyAsyncProcess(createHConnection(), conf); - Runnable runnable = new Runnable() { - @Override - public void run() { - try { - barrier.await(); - ap.waitForMaximumCurrentTasks(max.get(), tasks, 1, null); - } catch (InterruptedIOException e) { - Assert.fail(e.getMessage()); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } catch (BrokenBarrierException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - }; - // First test that our runnable thread only exits when tasks is zero. - Thread t = new Thread(runnable); - t.start(); - barrier.await(); - t.join(); - // Now assert we stay running if max == zero and tasks is > 0. - barrier.reset(); - tasks.set(1000000); - t = new Thread(runnable); - t.start(); - barrier.await(); - while (tasks.get() > 0) { - assertTrue(t.isAlive()); - tasks.set(tasks.get() - 1); - } - t.join(); - } - /** * Test and make sure we could use a special pause setting when retry with * CallQueueTooBigException, see HBASE-17114 @@ -1738,18 +1632,18 @@ public class TestAsyncProcess { */ @Test public void testRetryPauseWithCallQueueTooBigException() throws Exception { - Configuration myConf = new Configuration(conf); + Configuration myConf = new Configuration(CONF); final long specialPause = 500L; final int retries = 1; myConf.setLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, specialPause); myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries); ClusterConnection conn = new MyConnectionImpl(myConf); - BufferedMutatorImpl mutator = (BufferedMutatorImpl) conn.getBufferedMutator(DUMMY_TABLE); AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, myConf, new CallQueueTooBigException()); - mutator.ap = ap; + BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - Assert.assertNotNull(mutator.ap.createServerErrorTracker()); + Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); Put p = createPut(1, true); mutator.mutate(p); @@ -1775,8 +1669,9 @@ public class TestAsyncProcess { final long normalPause = myConf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE); ap = new AsyncProcessWithFailure(conn, myConf, new IOException()); - mutator.ap = ap; - Assert.assertNotNull(mutator.ap.createServerErrorTracker()); + bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); + mutator = new BufferedMutatorImpl(conn, bufferParam, ap); + Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); mutator.mutate(p); startTime = System.currentTimeMillis(); try { diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java new file mode 100644 index 0000000..b46e572 --- /dev/null +++ hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java @@ -0,0 +1,336 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RequestController.ReturnCode; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ClientTests.class, SmallTests.class}) +public class TestSimpleRequestController { + + private static final TableName DUMMY_TABLE + = TableName.valueOf("DUMMY_TABLE"); + private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes(); + private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes(); + private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes(); + private static final ServerName SN = ServerName.valueOf("s1:1,1"); + private static final ServerName SN2 = ServerName.valueOf("s2:2,2"); + private static final ServerName SN3 = ServerName.valueOf("s3:3,3"); + private static final HRegionInfo HRI1 + = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1); + private static final HRegionInfo HRI2 + = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_2, HConstants.EMPTY_END_ROW, false, 2); + private static final HRegionInfo HRI3 + = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_3, HConstants.EMPTY_END_ROW, false, 3); + private static final HRegionLocation LOC1 = new HRegionLocation(HRI1, SN); + private static final HRegionLocation LOC2 = new HRegionLocation(HRI2, SN); + private static final HRegionLocation LOC3 = new HRegionLocation(HRI3, SN2); + + @Test + public void testIllegalRequestSize() { + testIllegalArgument(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, -1); + } + + @Test + public void testIllegalRsTasks() { + testIllegalArgument(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS, -1); + } + + @Test + public void testIllegalRegionTasks() { + testIllegalArgument(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS, -1); + } + + @Test + public void testIllegalSubmittedSize() { + testIllegalArgument(SimpleRequestController.HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, -1); + } + + private void testIllegalArgument(String key, long value) { + Configuration conf = HBaseConfiguration.create(); + conf.setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, -1); + try { + SimpleRequestController controller = new SimpleRequestController(conf); + fail("The " + key + " must be bigger than zero"); + } catch (IllegalArgumentException e) { + } + } + + private static Put createPut(long maxHeapSizePerRequest) { + return new Put(Bytes.toBytes("row")) { + @Override + public long heapSize() { + return maxHeapSizePerRequest; + } + }; + } + + @Test + public void testTaskCheckerHost() throws IOException { + final int maxTotalConcurrentTasks = 100; + final int maxConcurrentTasksPerServer = 2; + final int maxConcurrentTasksPerRegion = 1; + final AtomicLong tasksInProgress = new AtomicLong(0); + final Map taskCounterPerServer = new HashMap<>(); + final Map taskCounterPerRegion = new HashMap<>(); + SimpleRequestController.TaskCountChecker countChecker = new SimpleRequestController.TaskCountChecker( + maxTotalConcurrentTasks, + maxConcurrentTasksPerServer, + maxConcurrentTasksPerRegion, + tasksInProgress, taskCounterPerServer, taskCounterPerRegion); + final long maxHeapSizePerRequest = 2 * 1024 * 1024; + // unlimiited + SimpleRequestController.RequestSizeChecker sizeChecker = new SimpleRequestController.RequestSizeChecker(maxHeapSizePerRequest); + RequestController.Checker checker = SimpleRequestController.newChecker(Arrays.asList(countChecker, sizeChecker)); + ReturnCode loc1Code = checker.canTakeRow(LOC1, createPut(maxHeapSizePerRequest)); + assertEquals(ReturnCode.INCLUDE, loc1Code); + + ReturnCode loc1Code_2 = checker.canTakeRow(LOC1, createPut(maxHeapSizePerRequest)); + // rejected for size + assertNotEquals(ReturnCode.INCLUDE, loc1Code_2); + + ReturnCode loc2Code = checker.canTakeRow(LOC2, createPut(maxHeapSizePerRequest)); + // rejected for size + assertNotEquals(ReturnCode.INCLUDE, loc2Code); + + // fill the task slots for LOC3. + taskCounterPerRegion.put(LOC3.getRegionInfo().getRegionName(), new AtomicInteger(100)); + taskCounterPerServer.put(LOC3.getServerName(), new AtomicInteger(100)); + + ReturnCode loc3Code = checker.canTakeRow(LOC3, createPut(1L)); + // rejected for count + assertNotEquals(ReturnCode.INCLUDE, loc3Code); + + // release the task slots for LOC3. + taskCounterPerRegion.put(LOC3.getRegionInfo().getRegionName(), new AtomicInteger(0)); + taskCounterPerServer.put(LOC3.getServerName(), new AtomicInteger(0)); + + ReturnCode loc3Code_2 = checker.canTakeRow(LOC3, createPut(1L)); + assertEquals(ReturnCode.INCLUDE, loc3Code_2); + } + + @Test + public void testRequestSizeCheckerr() throws IOException { + final long maxHeapSizePerRequest = 2 * 1024 * 1024; + SimpleRequestController.RequestSizeChecker checker + = new SimpleRequestController.RequestSizeChecker(maxHeapSizePerRequest); + + // inner state is unchanged. + for (int i = 0; i != 10; ++i) { + ReturnCode code = checker.canTakeOperation(LOC1, maxHeapSizePerRequest); + assertEquals(ReturnCode.INCLUDE, code); + code = checker.canTakeOperation(LOC2, maxHeapSizePerRequest); + assertEquals(ReturnCode.INCLUDE, code); + } + + // accept the data located on LOC1 region. + ReturnCode acceptCode = checker.canTakeOperation(LOC1, maxHeapSizePerRequest); + assertEquals(ReturnCode.INCLUDE, acceptCode); + checker.notifyFinal(acceptCode, LOC1, maxHeapSizePerRequest); + + // the sn server reachs the limit. + for (int i = 0; i != 10; ++i) { + ReturnCode code = checker.canTakeOperation(LOC1, maxHeapSizePerRequest); + assertNotEquals(ReturnCode.INCLUDE, code); + code = checker.canTakeOperation(LOC2, maxHeapSizePerRequest); + assertNotEquals(ReturnCode.INCLUDE, code); + } + + // the request to sn2 server should be accepted. + for (int i = 0; i != 10; ++i) { + ReturnCode code = checker.canTakeOperation(LOC3, maxHeapSizePerRequest); + assertEquals(ReturnCode.INCLUDE, code); + } + + checker.reset(); + for (int i = 0; i != 10; ++i) { + ReturnCode code = checker.canTakeOperation(LOC1, maxHeapSizePerRequest); + assertEquals(ReturnCode.INCLUDE, code); + code = checker.canTakeOperation(LOC2, maxHeapSizePerRequest); + assertEquals(ReturnCode.INCLUDE, code); + } + } + + @Test + public void testSubmittedSizeChecker() { + final long maxHeapSizeSubmit = 2 * 1024 * 1024; + SimpleRequestController.SubmittedSizeChecker checker + = new SimpleRequestController.SubmittedSizeChecker(maxHeapSizeSubmit); + + for (int i = 0; i != 10; ++i) { + ReturnCode include = checker.canTakeOperation(LOC1, 100000); + assertEquals(ReturnCode.INCLUDE, include); + } + + for (int i = 0; i != 10; ++i) { + checker.notifyFinal(ReturnCode.INCLUDE, LOC1, maxHeapSizeSubmit); + } + + for (int i = 0; i != 10; ++i) { + ReturnCode include = checker.canTakeOperation(LOC1, 100000); + assertEquals(ReturnCode.END, include); + } + for (int i = 0; i != 10; ++i) { + ReturnCode include = checker.canTakeOperation(LOC2, 100000); + assertEquals(ReturnCode.END, include); + } + checker.reset(); + for (int i = 0; i != 10; ++i) { + ReturnCode include = checker.canTakeOperation(LOC1, 100000); + assertEquals(ReturnCode.INCLUDE, include); + } + } + + @Test + public void testTaskCountChecker() throws InterruptedIOException { + long rowSize = 12345; + int maxTotalConcurrentTasks = 100; + int maxConcurrentTasksPerServer = 2; + int maxConcurrentTasksPerRegion = 1; + AtomicLong tasksInProgress = new AtomicLong(0); + Map taskCounterPerServer = new HashMap<>(); + Map taskCounterPerRegion = new HashMap<>(); + SimpleRequestController.TaskCountChecker checker = new SimpleRequestController.TaskCountChecker( + maxTotalConcurrentTasks, + maxConcurrentTasksPerServer, + maxConcurrentTasksPerRegion, + tasksInProgress, taskCounterPerServer, taskCounterPerRegion); + + // inner state is unchanged. + for (int i = 0; i != 10; ++i) { + ReturnCode code = checker.canTakeOperation(LOC1, rowSize); + assertEquals(ReturnCode.INCLUDE, code); + } + // add LOC1 region. + ReturnCode code = checker.canTakeOperation(LOC1, rowSize); + assertEquals(ReturnCode.INCLUDE, code); + checker.notifyFinal(code, LOC1, rowSize); + + // fill the task slots for LOC1. + taskCounterPerRegion.put(LOC1.getRegionInfo().getRegionName(), new AtomicInteger(100)); + taskCounterPerServer.put(LOC1.getServerName(), new AtomicInteger(100)); + + // the region was previously accepted, so it must be accpted now. + for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { + ReturnCode includeCode = checker.canTakeOperation(LOC1, rowSize); + assertEquals(ReturnCode.INCLUDE, includeCode); + checker.notifyFinal(includeCode, LOC1, rowSize); + } + + // fill the task slots for LOC3. + taskCounterPerRegion.put(LOC3.getRegionInfo().getRegionName(), new AtomicInteger(100)); + taskCounterPerServer.put(LOC3.getServerName(), new AtomicInteger(100)); + + // no task slots. + for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { + ReturnCode excludeCode = checker.canTakeOperation(LOC3, rowSize); + assertNotEquals(ReturnCode.INCLUDE, excludeCode); + checker.notifyFinal(excludeCode, LOC3, rowSize); + } + + // release the tasks for LOC3. + taskCounterPerRegion.put(LOC3.getRegionInfo().getRegionName(), new AtomicInteger(0)); + taskCounterPerServer.put(LOC3.getServerName(), new AtomicInteger(0)); + + // add LOC3 region. + ReturnCode code3 = checker.canTakeOperation(LOC3, rowSize); + assertEquals(ReturnCode.INCLUDE, code3); + checker.notifyFinal(code3, LOC3, rowSize); + + // the region was previously accepted, so it must be accpted now. + for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { + ReturnCode includeCode = checker.canTakeOperation(LOC3, rowSize); + assertEquals(ReturnCode.INCLUDE, includeCode); + checker.notifyFinal(includeCode, LOC3, rowSize); + } + + checker.reset(); + // the region was previously accepted, + // but checker have reseted and task slots for LOC1 is full. + // So it must be rejected now. + for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) { + ReturnCode includeCode = checker.canTakeOperation(LOC1, rowSize); + assertNotEquals(ReturnCode.INCLUDE, includeCode); + checker.notifyFinal(includeCode, LOC1, rowSize); + } + } + + @Test + public void testWaitForMaximumCurrentTasks() throws Exception { + final AtomicInteger max = new AtomicInteger(0); + final CyclicBarrier barrier = new CyclicBarrier(2); + SimpleRequestController controller = new SimpleRequestController(HBaseConfiguration.create()); + final AtomicLong tasks = controller.tasksInProgress; + Runnable runnable = () -> { + try { + barrier.await(); + controller.waitForMaximumCurrentTasks(max.get(), 123, 1, null); + } catch (InterruptedIOException e) { + Assert.fail(e.getMessage()); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } catch (BrokenBarrierException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + }; + // First test that our runnable thread only exits when tasks is zero. + Thread t = new Thread(runnable); + t.start(); + barrier.await(); + t.join(); + // Now assert we stay running if max == zero and tasks is > 0. + barrier.reset(); + tasks.set(1000000); + t = new Thread(runnable); + t.start(); + barrier.await(); + while (tasks.get() > 0) { + assertTrue(t.isAlive()); + tasks.set(tasks.get() - 1); + } + t.join(); + } +} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index ee89609..e5ab3e8 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -126,11 +126,8 @@ public class HConnectionTestingUtility { 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, - RpcControllerFactory.instantiate(conf), conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT), conf.getInt( - HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT))); + new AsyncProcess(c, conf, RpcRetryingCallerFactory.instantiate(conf), false, + RpcControllerFactory.instantiate(conf))); Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn( RpcRetryingCallerFactory.instantiate(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null)); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java index 53488ec..2c5e89d 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.AsyncProcessTask; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ServerStatistics; @@ -137,14 +138,20 @@ public class TestClientPushback { final CountDownLatch latch = new CountDownLatch(1); final AtomicLong endTime = new AtomicLong(); long startTime = EnvironmentEdgeManager.currentTime(); - - ((HTable) table).mutator.ap.submit(null, tableName, ops, true, new Batch.Callback() { - @Override - public void update(byte[] region, byte[] row, Result result) { + BufferedMutatorImpl mutator = ((HTable) table).mutator; + Batch.Callback callback = (byte[] r, byte[] row, Result result) -> { endTime.set(EnvironmentEdgeManager.currentTime()); latch.countDown(); - } - }, true); + }; + AsyncProcessTask task = AsyncProcessTask.newBuilder(callback) + .setPool(mutator.getPool()) + .setTableName(tableName) + .setRowAccess(ops) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.AT_LEAST_ONE) + .setOperationTimeout(conn.getConnectionConfiguration().getOperationTimeout()) + .setRpcTimeout(60 * 1000) + .build(); + mutator.getAsyncProcess().submit(task); // Currently the ExponentialClientBackoffPolicy under these test conditions // produces a backoffTime of 151 milliseconds. This is long enough so the // wait and related checks below are reasonable. Revisit if the backoff diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index 6d1e1f0..0f7f3d9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -563,9 +563,17 @@ public class TestReplicasClient { gets.add(g); Object[] results = new Object[2]; - AsyncRequestFuture reqs = ap.submitAll( - HTable.getDefaultExecutor(HTU.getConfiguration()), - table.getName(), gets, null, results); + int operationTimeout = ((ClusterConnection) HTU.getConnection()).getConnectionConfiguration().getOperationTimeout(); + int readTimeout = ((ClusterConnection) HTU.getConnection()).getConnectionConfiguration().getReadRpcTimeout(); + AsyncProcessTask task = AsyncProcessTask.newBuilder() + .setPool(HTable.getDefaultExecutor(HTU.getConfiguration())) + .setTableName(table.getName()) + .setRowAccess(gets) + .setResults(results) + .setOperationTimeout(operationTimeout) + .setRpcTimeout(readTimeout) + .build(); + AsyncRequestFuture reqs = ap.submit(task); reqs.waitUntilDone(); // verify we got the right results back for (Object r : results) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java index be41e54..295f47a 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java @@ -501,7 +501,6 @@ public class TestPerColumnFamilyFlush { Thread.sleep(100); } } - table.close(); assertEquals(maxLogs, getNumRolledLogFiles(desiredRegion)); assertTrue(desiredRegion.getStore(FAMILY1).getMemStoreSize() > cfFlushSizeLowerBound); assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize() < cfFlushSizeLowerBound); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java index 68fffb1..380c252 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java @@ -171,22 +171,35 @@ public class TestTablePermissions { } } + /** + * The AccessControlLists.addUserPermission may throw exception before closing the table. + */ + private void addUserPermission(Configuration conf, UserPermission userPerm, Table t) throws IOException { + try { + AccessControlLists.addUserPermission(conf, userPerm, t); + } finally { + t.close(); + } + } + @Test public void testBasicWrite() throws Exception { Configuration conf = UTIL.getConfiguration(); - try (Connection connection = ConnectionFactory.createConnection(conf); - Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) { + try (Connection connection = ConnectionFactory.createConnection(conf)) { // add some permissions - AccessControlLists.addUserPermission(conf, + addUserPermission(conf, new UserPermission(Bytes.toBytes("george"), TEST_TABLE, null, (byte[])null, - UserPermission.Action.READ, UserPermission.Action.WRITE), table); - AccessControlLists.addUserPermission(conf, + UserPermission.Action.READ, UserPermission.Action.WRITE), + connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("hubert"), TEST_TABLE, null, (byte[])null, - UserPermission.Action.READ), table); - AccessControlLists.addUserPermission(conf, + UserPermission.Action.READ), + connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("humphrey"), TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, - UserPermission.Action.READ), table); + UserPermission.Action.READ), + connection.getTable(AccessControlLists.ACL_TABLE_NAME)); } // retrieve the same ListMultimap perms = @@ -274,23 +287,22 @@ public class TestTablePermissions { @Test public void testPersistence() throws Exception { Configuration conf = UTIL.getConfiguration(); - try (Connection connection = ConnectionFactory.createConnection(conf); - Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) { - AccessControlLists.addUserPermission(conf, + try (Connection connection = ConnectionFactory.createConnection(conf)) { + addUserPermission(conf, new UserPermission(Bytes.toBytes("albert"), TEST_TABLE, null, - (byte[])null, TablePermission.Action.READ), table); - AccessControlLists.addUserPermission(conf, + (byte[])null, TablePermission.Action.READ), connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("betty"), TEST_TABLE, null, (byte[])null, TablePermission.Action.READ, - TablePermission.Action.WRITE), table); - AccessControlLists.addUserPermission(conf, + TablePermission.Action.WRITE), connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("clark"), TEST_TABLE, TEST_FAMILY, - TablePermission.Action.READ), table); - AccessControlLists.addUserPermission(conf, + TablePermission.Action.READ), connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("dwight"), TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, - TablePermission.Action.WRITE), table); + TablePermission.Action.WRITE), connection.getTable(AccessControlLists.ACL_TABLE_NAME)); } // verify permissions survive changes in table metadata ListMultimap preperms = @@ -404,17 +416,17 @@ public class TestTablePermissions { Configuration conf = UTIL.getConfiguration(); // add some permissions - try (Connection connection = ConnectionFactory.createConnection(conf); - Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) { - AccessControlLists.addUserPermission(conf, + try (Connection connection = ConnectionFactory.createConnection(conf)) { + addUserPermission(conf, new UserPermission(Bytes.toBytes("user1"), - Permission.Action.READ, Permission.Action.WRITE), table); - AccessControlLists.addUserPermission(conf, + Permission.Action.READ, Permission.Action.WRITE), connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("user2"), - Permission.Action.CREATE), table); - AccessControlLists.addUserPermission(conf, + Permission.Action.CREATE), connection.getTable(AccessControlLists.ACL_TABLE_NAME)); + addUserPermission(conf, new UserPermission(Bytes.toBytes("user3"), - Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE), table); + Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE), + connection.getTable(AccessControlLists.ACL_TABLE_NAME)); } ListMultimap perms = AccessControlLists.getTablePermissions(conf, null); List user1Perms = perms.get("user1"); @@ -448,11 +460,11 @@ public class TestTablePermissions { // currently running user is the system user and should have global admin perms User currentUser = User.getCurrent(); assertTrue(authManager.authorize(currentUser, Permission.Action.ADMIN)); - try (Connection connection = ConnectionFactory.createConnection(conf); - Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) { + try (Connection connection = ConnectionFactory.createConnection(conf)) { for (int i=1; i<=50; i++) { - AccessControlLists.addUserPermission(conf, new UserPermission(Bytes.toBytes("testauth"+i), - Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.WRITE), table); + addUserPermission(conf, new UserPermission(Bytes.toBytes("testauth"+i), + Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.WRITE), + connection.getTable(AccessControlLists.ACL_TABLE_NAME)); // make sure the system user still shows as authorized assertTrue("Failed current user auth check on iter "+i, authManager.authorize(currentUser, Permission.Action.ADMIN));