diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java index 22042c9b8314..52441b90a372 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionConfiguration.java @@ -38,8 +38,6 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_READ_TIMEOUT_KEY; import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY; import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY; -import static org.apache.hadoop.hbase.client.AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.apache.hadoop.hbase.client.ConnectionConfiguration.MAX_KEYVALUE_SIZE_DEFAULT; import static org.apache.hadoop.hbase.client.ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY; import static org.apache.hadoop.hbase.client.ConnectionConfiguration.PRIMARY_CALL_TIMEOUT_MICROSECOND; @@ -61,6 +59,16 @@ @InterfaceAudience.Private class AsyncConnectionConfiguration { + /** + * Configure the number of failures after which the client will start logging. A few failures + * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable + * heuristic for the number of errors we don't log. 5 was chosen because we wait for 1s at + * this stage. + */ + public static final String START_LOG_ERRORS_AFTER_COUNT_KEY = + "hbase.client.start.log.errors.counter"; + public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 5; + private final long metaOperationTimeoutNs; // timeout for a whole operation such as get, put or delete. Notice that scan will not be effected diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java index 9cf8bc651d66..ceb0a3c7368b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + /** * The asynchronous locator for meta region. */ @@ -133,4 +135,17 @@ void clearCache(ServerName serverName) { } } } + + // only used for testing whether we have cached the location for a region. + @VisibleForTesting + RegionLocations getRegionLocationInCache() { + return metaRegionLocations.get(); + } + + // only used for testing whether we have cached the location for a table. + @VisibleForTesting + int getNumberOfCachedRegionLocations() { + RegionLocations locs = metaRegionLocations.get(); + return locs != null ? locs.numNonNullElements() : 0; + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index 4a9b7dcce321..5ab023d60a7a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java @@ -654,4 +654,14 @@ RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) { } return locateRowInCache(tableCache, tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID); } + + // only used for testing whether we have cached the location for a table. + @VisibleForTesting + int getNumberOfCachedRegionLocations(TableName tableName) { + TableCache tableCache = cache.get(tableName); + if (tableCache == null) { + return 0; + } + return tableCache.cache.values().stream().mapToInt(RegionLocations::numNonNullElements).sum(); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java deleted file mode 100644 index b0f863f47458..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ /dev/null @@ -1,482 +0,0 @@ -/* - * - * 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.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; -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.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -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; - -/** - * This class allows a continuous flow of requests. It's written to be compatible with a - * synchronous caller such as HTable. - *

- * The caller sends a buffer of operation, by calling submit. This class extract from this list - * the operations it can send, i.e. the operations that are on region that are not considered - * as busy. The process is asynchronous, i.e. it returns immediately when if has finished to - * iterate on the list. If, and only if, the maximum number of current task is reached, the call - * to submit will block. Alternatively, the caller can call submitAll, in which case all the - * operations will be sent. Each call to submit returns a future-like object that can be used - * to track operation progress. - *

- *

- * The class manages internally the retries. - *

- *

- * The errors are tracked inside the Future object that is returned. - * The results are always tracked inside the Future object and can be retrieved when the call - * has finished. Partial results can also be retrieved if some part of multi-request failed. - *

- *

- * This class is thread safe. - * Internally, the class is thread safe enough to manage simultaneously new submission and results - * arising from older operations. - *

- *

- * Internally, this class works with {@link Row}, this mean it could be theoretically used for - * gets as well. - *

- */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -class AsyncProcess { - private static final Logger LOG = LoggerFactory.getLogger(AsyncProcess.class); - private static final AtomicLong COUNTER = new AtomicLong(); - - public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget"; - - /** - * Configure the number of failures after which the client will start logging. A few failures - * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable - * heuristic for the number of errors we don't log. 5 was chosen because we wait for 1s at - * this stage. - */ - public static final String START_LOG_ERRORS_AFTER_COUNT_KEY = - "hbase.client.start.log.errors.counter"; - public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 5; - - /** - * Configuration to decide whether to log details for batch error - */ - public static final String LOG_DETAILS_FOR_BATCH_ERROR = "hbase.client.log.batcherrors.details"; - - /** - * Return value from a submit that didn't contain any requests. - */ - private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() { - final Object[] result = new Object[0]; - - @Override - public boolean hasError() { - return false; - } - - @Override - public RetriesExhaustedWithDetailsException getErrors() { - return null; - } - - @Override - public List getFailedOperations() { - return null; - } - - @Override - public Object[] getResults() { - return result; - } - - @Override - public void waitUntilDone() throws InterruptedIOException { - } - }; - - // TODO: many of the fields should be made private - final long id; - - final ConnectionImplementation connection; - private final RpcRetryingCallerFactory rpcCallerFactory; - final RpcControllerFactory rpcFactory; - - // Start configuration settings. - final int startLogErrorsCnt; - - final long pause; - final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified - final int numTries; - @VisibleForTesting - long serverTrackerTimeout; - final long primaryCallTimeoutMicroseconds; - /** Whether to log details for batch errors */ - final boolean logBatchErrorDetails; - // End configuration settings. - - /** - * 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 periodToLog; - AsyncProcess(ConnectionImplementation hc, Configuration conf, - RpcRetryingCallerFactory rpcCaller, RpcControllerFactory rpcFactory) { - if (hc == null) { - throw new IllegalArgumentException("ClusterConnection cannot be null."); - } - - this.connection = hc; - - this.id = COUNTER.incrementAndGet(); - - this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); - if (configuredPauseForCQTBE < pause) { - LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " - + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE - + ", will use " + pause + " instead."); - this.pauseForCQTBE = pause; - } else { - this.pauseForCQTBE = configuredPauseForCQTBE; - } - // 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.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000); - this.startLogErrorsCnt = - conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT); - this.periodToLog = 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 - // retry time if normal retries were used. Then we will retry until this time runs out. - // If we keep hitting one server, the net effect will be the incremental backoff, and - // essentially the same number of retries as planned. If we have to do faster retries, - // we will do more retries in aggregate, but the user will be none the wiser. - this.serverTrackerTimeout = 0L; - for (int i = 0; i < this.numTries; ++i) { - serverTrackerTimeout = serverTrackerTimeout + ConnectionUtils.getPauseTime(this.pause, i); - } - - this.rpcCallerFactory = rpcCaller; - this.rpcFactory = rpcFactory; - this.logBatchErrorDetails = conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false); - - this.requestController = RequestControllerFactory.create(conf); - } - - /** - * 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 - */ - 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); - } - } - - /** - * Extract from the rows list what we can submit. The rows we can not submit are kept in the - * list. Does not send requests to replicas (not currently used for anything other - * than streaming puts anyway). - * - * @param task The setting and data - * @param atLeastOne true if we should submit at least a subset. - */ - 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()); - - NonceGenerator ng = this.connection.getNonceGenerator(); - long nonceGroup = ng.getNonceGroup(); // Currently, nonce group is per entire client. - - // Location errors that happen before we decide what requests to take. - List locationErrors = null; - List locationErrorRows = null; - RequestController.Checker checker = requestController.newChecker(); - boolean firstIter = true; - do { - // Wait until there is at least one slot for a new task. - requestController.waitForFreeSlot(id, periodToLog, getLogger(tableName, -1)); - int posInList = -1; - if (!firstIter) { - checker.reset(); - } - Iterator it = rows.iterator(); - while (it.hasNext()) { - Row r = it.next(); - HRegionLocation loc; - try { - if (r == null) { - throw new IllegalArgumentException("#" + id + ", row cannot be null"); - } - // Make sure we get 0-s replica. - RegionLocations locs = connection.locateRegion( - tableName, r.getRow(), true, true, RegionReplicaUtil.DEFAULT_REPLICA_ID); - if (locs == null || locs.isEmpty() || locs.getDefaultRegionLocation() == null) { - throw new IOException("#" + id + ", no location found, aborting submit for" - + " tableName=" + tableName + " rowkey=" + Bytes.toStringBinary(r.getRow())); - } - loc = locs.getDefaultRegionLocation(); - } catch (IOException ex) { - locationErrors = new ArrayList<>(1); - locationErrorRows = new ArrayList<>(1); - LOG.error("Failed to get region location ", ex); - // This action failed before creating ars. Retain it, but do not add to submit list. - // We will then add it to ars in an already-failed state. - - int priority = HConstants.NORMAL_QOS; - if (r instanceof Mutation) { - priority = ((Mutation) r).getPriority(); - } - retainedActions.add(new Action(r, ++posInList, priority)); - locationErrors.add(ex); - locationErrorRows.add(posInList); - it.remove(); - break; // Backward compat: we stop considering actions on location error. - } - ReturnCode code = checker.canTakeRow(loc, r); - if (code == ReturnCode.END) { - break; - } - if (code == ReturnCode.INCLUDE) { - int priority = HConstants.NORMAL_QOS; - if (r instanceof Mutation) { - priority = ((Mutation) r).getPriority(); - } - Action action = new Action(r, ++posInList, priority); - setNonce(ng, r, action); - retainedActions.add(action); - // TODO: replica-get is not supported on this path - byte[] regionName = loc.getRegionInfo().getRegionName(); - addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup); - it.remove(); - } - } - firstIter = false; - } while (retainedActions.isEmpty() && atLeastOne && (locationErrors == null)); - - if (retainedActions.isEmpty()) return NO_REQS_RESULT; - - return submitMultiActions(task, retainedActions, nonceGroup, - locationErrors, locationErrorRows, actionsByServer); - } - - 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) { - int originalIndex = locationErrorRows.get(i); - Row row = retainedActions.get(originalIndex).getAction(); - ars.manageError(originalIndex, row, - AsyncRequestFutureImpl.Retry.NO_LOCATION_PROBLEM, locationErrors.get(i), null); - } - } - ars.sendMultiAction(actionsByServer, 1, null, false); - return ars; - } - - /** - * Helper that is used when grouping the actions per region server. - * - * @param server - server - * @param regionName - regionName - * @param action - the action to add to the multiaction - * @param actionsByServer the multiaction per server - * @param nonceGroup Nonce group. - */ - static void addAction(ServerName server, byte[] regionName, Action action, - Map actionsByServer, long nonceGroup) { - MultiAction multiAction = actionsByServer.get(server); - if (multiAction == null) { - multiAction = new MultiAction(); - actionsByServer.put(server, multiAction); - } - if (action.hasNonce() && !multiAction.hasNonceGroup()) { - multiAction.setNonceGroup(nonceGroup); - } - - multiAction.add(regionName, action); - } - - /** - * 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. - * @param task The setting and data - */ - 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. - int posInList = -1; - NonceGenerator ng = this.connection.getNonceGenerator(); - int highestPriority = HConstants.PRIORITY_UNSET; - for (Row r : rows) { - posInList++; - if (r instanceof Put) { - Put put = (Put) r; - if (put.isEmpty()) { - throw new IllegalArgumentException("No columns to insert for #" + (posInList+1)+ " item"); - } - highestPriority = Math.max(put.getPriority(), highestPriority); - } - Action action = new Action(r, posInList, highestPriority); - setNonce(ng, r, action); - actions.add(action); - } - 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. - } - - 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, TableName tableName) - throws InterruptedIOException { - requestController.waitForMaximumCurrentTasks(max, id, periodToLog, - getLogger(tableName, max)); - } - - 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 + (tableName == null ? "" : ", tableName=" + tableName)); - }; - } - - void incTaskCounters(Collection regions, ServerName sn) { - requestController.incTaskCounters(regions, sn); - } - - - void decTaskCounters(Collection regions, ServerName sn) { - requestController.decTaskCounters(regions, sn); - } - - /** - * Create a caller. Isolated to be easily overridden in the tests. - */ - @VisibleForTesting - protected RpcRetryingCaller createCaller( - CancellableRegionServerCallable callable, int rpcTimeout) { - return rpcCallerFactory. newCaller(checkRpcTimeout(rpcTimeout)); - } - - - /** - * Creates the server error tracker to use inside process. - * Currently, to preserve the main assumption about current retries, and to work well with - * the retry-limit-based calculation, the calculation is local per Process object. - * We may benefit from connection-wide tracking of server errors. - * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection - */ - ConnectionImplementation.ServerErrorTracker createServerErrorTracker() { - return new ConnectionImplementation.ServerErrorTracker( - this.serverTrackerTimeout, this.numTries); - } - - static boolean isReplicaGet(Row row) { - return (row instanceof Get) && (((Get)row).getConsistency() == Consistency.TIMELINE); - } - -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java deleted file mode 100644 index 5a2bbfebc08d..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * - * 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.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.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 a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java index 9e1d5e8f5a65..09eabfc1d53f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java @@ -179,4 +179,24 @@ void clearCache() { AsyncNonMetaRegionLocator getNonMetaRegionLocator() { return nonMetaRegionLocator; } + + // only used for testing whether we have cached the location for a region. + @VisibleForTesting + RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) { + if (TableName.isMetaTableName(tableName)) { + return metaRegionLocator.getRegionLocationInCache(); + } else { + return nonMetaRegionLocator.getRegionLocationInCache(tableName, row); + } + } + + // only used for testing whether we have cached the location for a table. + @VisibleForTesting + int getNumberOfCachedRegionLocations(TableName tableName) { + if (TableName.isMetaTableName(tableName)) { + return metaRegionLocator.getNumberOfCachedRegionLocations(); + } else { + return nonMetaRegionLocator.getNumberOfCachedRegionLocations(tableName); + } + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java deleted file mode 100644 index b91e094d340d..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFuture.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * - * 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.yetus.audience.InterfaceAudience; - -import java.io.InterruptedIOException; -import java.util.List; - -/** - * The context used to wait for results from one submit call. If submit call is made with - * needResults false, results will not be saved. - * @since 2.0.0 - */ -@InterfaceAudience.Private -public interface AsyncRequestFuture { - public boolean hasError(); - public RetriesExhaustedWithDetailsException getErrors(); - public List getFailedOperations(); - public Object[] getResults() throws InterruptedIOException; - /** Wait until all tasks are executed, successfully or not. */ - public void waitUntilDone() throws InterruptedIOException; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java deleted file mode 100644 index e46a50e76199..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java +++ /dev/null @@ -1,1249 +0,0 @@ -/* - * - * 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.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hbase.CallQueueTooBigException; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.RetryImmediatelyException; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.backoff.ServerStatistics; -import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; -import org.apache.hadoop.hbase.trace.TraceUtil; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.htrace.core.Tracer; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -/** - * The context, and return value, for a single submit/submitAll call. - * Note on how this class (one AP submit) works. Initially, all requests are split into groups - * by server; request is sent to each server in parallel; the RPC calls are not async so a - * thread per server is used. Every time some actions fail, regions/locations might have - * changed, so we re-group them by server and region again and send these groups in parallel - * too. The result, in case of retries, is a "tree" of threads, with parent exiting after - * scheduling children. This is why lots of code doesn't require any synchronization. - */ -@InterfaceAudience.Private -class AsyncRequestFutureImpl implements AsyncRequestFuture { - - private static final Logger LOG = LoggerFactory.getLogger(AsyncRequestFutureImpl.class); - - private RetryingTimeTracker tracker; - - /** - * Runnable (that can be submitted to thread pool) that waits for when it's time - * to issue replica calls, finds region replicas, groups the requests by replica and - * issues the calls (on separate threads, via sendMultiAction). - * This is done on a separate thread because we don't want to wait on user thread for - * our asynchronous call, and usually we have to wait before making replica calls. - */ - private final class ReplicaCallIssuingRunnable implements Runnable { - private final long startTime; - private final List initialActions; - - public ReplicaCallIssuingRunnable(List initialActions, long startTime) { - this.initialActions = initialActions; - this.startTime = startTime; - } - - @Override - public void run() { - boolean done = false; - if (asyncProcess.primaryCallTimeoutMicroseconds > 0) { - try { - done = waitUntilDone(startTime * 1000L + asyncProcess.primaryCallTimeoutMicroseconds); - } catch (InterruptedException ex) { - LOG.error("Replica thread interrupted - no replica calls {}", ex.getMessage()); - return; - } - } - if (done) return; // Done within primary timeout - Map actionsByServer = new HashMap<>(); - List unknownLocActions = new ArrayList<>(); - if (replicaGetIndices == null) { - for (int i = 0; i < results.length; ++i) { - addReplicaActions(i, actionsByServer, unknownLocActions); - } - } else { - for (int replicaGetIndice : replicaGetIndices) { - addReplicaActions(replicaGetIndice, actionsByServer, unknownLocActions); - } - } - if (!actionsByServer.isEmpty()) { - sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty()); - } - if (!unknownLocActions.isEmpty()) { - actionsByServer = new HashMap<>(); - for (Action action : unknownLocActions) { - addReplicaActionsAgain(action, actionsByServer); - } - // Some actions may have completely failed, they are handled inside addAgain. - if (!actionsByServer.isEmpty()) { - sendMultiAction(actionsByServer, 1, null, true); - } - } - } - - /** - * Add replica actions to action map by server. - * @param index Index of the original action. - * @param actionsByServer The map by server to add it to. - */ - private void addReplicaActions(int index, Map actionsByServer, - List unknownReplicaActions) { - if (results[index] != null) return; // opportunistic. Never goes from non-null to null. - Action action = initialActions.get(index); - RegionLocations loc = findAllLocationsOrFail(action, true); - if (loc == null) return; - HRegionLocation[] locs = loc.getRegionLocations(); - if (locs.length == 1) { - LOG.warn("No replicas found for {}", action.getAction()); - return; - } - synchronized (replicaResultLock) { - // Don't run replica calls if the original has finished. We could do it e.g. if - // original has already failed before first replica call (unlikely given retries), - // but that would require additional synchronization w.r.t. returning to caller. - if (results[index] != null) return; - // We set the number of calls here. After that any path must call setResult/setError. - // True even for replicas that are not found - if we refuse to send we MUST set error. - updateResult(index, new ReplicaResultState(locs.length)); - } - for (int i = 1; i < locs.length; ++i) { - Action replicaAction = new Action(action, i); - if (locs[i] != null) { - asyncProcess.addAction(locs[i].getServerName(), locs[i].getRegionInfo().getRegionName(), - replicaAction, actionsByServer, nonceGroup); - } else { - unknownReplicaActions.add(replicaAction); - } - } - } - - private void addReplicaActionsAgain( - Action action, Map actionsByServer) { - if (action.getReplicaId() == RegionReplicaUtil.DEFAULT_REPLICA_ID) { - throw new AssertionError("Cannot have default replica here"); - } - HRegionLocation loc = getReplicaLocationOrFail(action); - if (loc == null) return; - asyncProcess.addAction(loc.getServerName(), loc.getRegionInfo().getRegionName(), - action, actionsByServer, nonceGroup); - } - } - - /** - * Runnable (that can be submitted to thread pool) that submits MultiAction to a - * single server. The server call is synchronous, therefore we do it on a thread pool. - */ - @VisibleForTesting - final class SingleServerRequestRunnable implements Runnable { - private final MultiAction multiAction; - private final int numAttempt; - private final ServerName server; - private final Set callsInProgress; - @VisibleForTesting - SingleServerRequestRunnable( - MultiAction multiAction, int numAttempt, ServerName server, - Set callsInProgress) { - this.multiAction = multiAction; - this.numAttempt = numAttempt; - this.server = server; - this.callsInProgress = callsInProgress; - } - - @Override - public void run() { - AbstractResponse res = null; - CancellableRegionServerCallable callable = currentCallable; - try { - // setup the callable based on the actions, if we don't have one already from the request - if (callable == null) { - callable = createCallable(server, tableName, multiAction); - } - RpcRetryingCaller caller = asyncProcess.createCaller(callable,rpcTimeout); - try { - if (callsInProgress != null) { - callsInProgress.add(callable); - } - res = caller.callWithoutRetries(callable, operationTimeout); - if (res == null) { - // Cancelled - return; - } - } catch (IOException e) { - // The service itself failed . It may be an error coming from the communication - // layer, but, as well, a functional error raised by the server. - receiveGlobalFailure(multiAction, server, numAttempt, e); - return; - } catch (Throwable t) { - // This should not happen. Let's log & retry anyway. - LOG.error("id=" + asyncProcess.id + ", caught throwable. Unexpected." + - " Retrying. Server=" + server + ", tableName=" + tableName, t); - receiveGlobalFailure(multiAction, server, numAttempt, t); - return; - } - if (res.type() == AbstractResponse.ResponseType.MULTI) { - // Normal case: we received an answer from the server, and it's not an exception. - receiveMultiAction(multiAction, server, (MultiResponse) res, numAttempt); - } else { - if (results != null) { - SingleResponse singleResponse = (SingleResponse) res; - updateResult(0, singleResponse.getEntry()); - } - decActionCounter(1); - } - } catch (Throwable t) { - // Something really bad happened. We are on the send thread that will now die. - LOG.error("id=" + asyncProcess.id + " error for " + tableName + " processing " + server, t); - throw new RuntimeException(t); - } finally { - asyncProcess.decTaskCounters(multiAction.getRegions(), server); - if (callsInProgress != null && callable != null && res != null) { - callsInProgress.remove(callable); - } - } - } - } - - private final Batch.Callback callback; - private final BatchErrors errors; - private final ConnectionImplementation.ServerErrorTracker errorsByServer; - private final ExecutorService pool; - private final Set callsInProgress; - - - private final TableName tableName; - private final AtomicLong actionsInProgress = new AtomicLong(-1); - /** - * The lock controls access to results. It is only held when populating results where - * there might be several callers (eventual consistency gets). For other requests, - * there's one unique call going on per result index. - */ - private final Object replicaResultLock = new Object(); - /** - * Result array. Null if results are not needed. Otherwise, each index corresponds to - * the action index in initial actions submitted. For most request types, has null-s for - * requests that are not done, and result/exception for those that are done. - * For eventual-consistency gets, initially the same applies; at some point, replica calls - * might be started, and ReplicaResultState is put at the corresponding indices. The - * returning calls check the type to detect when this is the case. After all calls are done, - * ReplicaResultState-s are replaced with results for the user. - */ - private final Object[] results; - /** - * Indices of replica gets in results. If null, all or no actions are replica-gets. - */ - private final int[] replicaGetIndices; - private final boolean hasAnyReplicaGets; - private final long nonceGroup; - private final CancellableRegionServerCallable currentCallable; - private final int operationTimeout; - private final int rpcTimeout; - private final AsyncProcess asyncProcess; - - /** - * For {@link AsyncRequestFutureImpl#manageError(int, Row, Retry, Throwable, ServerName)}. Only - * used to make logging more clear, we don't actually care why we don't retry. - */ - public enum Retry { - YES, - NO_LOCATION_PROBLEM, - NO_NOT_RETRIABLE, - NO_RETRIES_EXHAUSTED, - NO_OTHER_SUCCEEDED - } - - /** Sync point for calls to multiple replicas for the same user request (Get). - * Created and put in the results array (we assume replica calls require results) when - * the replica calls are launched. See results for details of this process. - * POJO, all fields are public. To modify them, the object itself is locked. */ - private static class ReplicaResultState { - public ReplicaResultState(int callCount) { - this.callCount = callCount; - } - - /** Number of calls outstanding, or 0 if a call succeeded (even with others outstanding). */ - int callCount; - /** Errors for which it is not decided whether we will report them to user. If one of the - * calls succeeds, we will discard the errors that may have happened in the other calls. */ - BatchErrors replicaErrors = null; - - @Override - public String toString() { - return "[call count " + callCount + "; errors " + replicaErrors + "]"; - } - } - - public AsyncRequestFutureImpl(AsyncProcessTask task, List actions, - long nonceGroup, AsyncProcess asyncProcess) { - this.pool = task.getPool(); - this.callback = task.getCallback(); - this.nonceGroup = nonceGroup; - this.tableName = task.getTableName(); - this.actionsInProgress.set(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 = task.getResults(); - for (int i = 0; i != this.results.length; ++i) { - results[i] = null; - } - } - List replicaGetIndices = null; - boolean hasAnyReplicaGets = false; - 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 - // store the list of action indexes for which replica gets are possible, and set - // hasAnyReplicaGets to true. - boolean hasAnyNonReplicaReqs = false; - int posInList = 0; - for (Action action : actions) { - boolean isReplicaGet = AsyncProcess.isReplicaGet(action.getAction()); - if (isReplicaGet) { - hasAnyReplicaGets = true; - if (hasAnyNonReplicaReqs) { // Mixed case - if (replicaGetIndices == null) { - replicaGetIndices = new ArrayList<>(actions.size() - 1); - } - replicaGetIndices.add(posInList); - } - } else if (!hasAnyNonReplicaReqs) { - // The first non-multi-replica request in the action list. - hasAnyNonReplicaReqs = true; - if (posInList > 0) { - // Add all the previous requests to the index lists. We know they are all - // replica-gets because this is the first non-multi-replica request in the list. - replicaGetIndices = new ArrayList<>(actions.size() - 1); - for (int i = 0; i < posInList; ++i) { - replicaGetIndices.add(i); - } - } - } - ++posInList; - } - } - this.hasAnyReplicaGets = hasAnyReplicaGets; - if (replicaGetIndices != null) { - this.replicaGetIndices = new int[replicaGetIndices.size()]; - int i = 0; - for (Integer el : replicaGetIndices) { - this.replicaGetIndices[i++] = el; - } - } else { - this.replicaGetIndices = null; - } - this.callsInProgress = !hasAnyReplicaGets ? null : - Collections.newSetFromMap( - new ConcurrentHashMap()); - this.asyncProcess = asyncProcess; - this.errorsByServer = createServerErrorTracker(); - this.errors = new BatchErrors(); - this.operationTimeout = task.getOperationTimeout(); - this.rpcTimeout = task.getRpcTimeout(); - this.currentCallable = task.getCallable(); - if (task.getCallable() == null) { - tracker = new RetryingTimeTracker().start(); - } - } - - @VisibleForTesting - protected Set getCallsInProgress() { - return callsInProgress; - } - - @VisibleForTesting - SingleServerRequestRunnable createSingleServerRequest(MultiAction multiAction, int numAttempt, ServerName server, - Set callsInProgress) { - return new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress); - } - - /** - * Group a list of actions per region servers, and send them. - * - * @param currentActions - the list of row to submit - * @param numAttempt - the current numAttempt (first attempt is 1) - */ - void groupAndSendMultiAction(List currentActions, int numAttempt) { - Map actionsByServer = new HashMap<>(); - - boolean isReplica = false; - List unknownReplicaActions = null; - for (Action action : currentActions) { - RegionLocations locs = findAllLocationsOrFail(action, true); - if (locs == null) continue; - boolean isReplicaAction = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId()); - if (isReplica && !isReplicaAction) { - // This is the property of the current implementation, not a requirement. - throw new AssertionError("Replica and non-replica actions in the same retry"); - } - isReplica = isReplicaAction; - HRegionLocation loc = locs.getRegionLocation(action.getReplicaId()); - if (loc == null || loc.getServerName() == null) { - if (isReplica) { - if (unknownReplicaActions == null) { - unknownReplicaActions = new ArrayList<>(1); - } - unknownReplicaActions.add(action); - } else { - // TODO: relies on primary location always being fetched - manageLocationError(action, null); - } - } else { - byte[] regionName = loc.getRegionInfo().getRegionName(); - AsyncProcess.addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup); - } - } - boolean doStartReplica = (numAttempt == 1 && !isReplica && hasAnyReplicaGets); - boolean hasUnknown = unknownReplicaActions != null && !unknownReplicaActions.isEmpty(); - - if (!actionsByServer.isEmpty()) { - // If this is a first attempt to group and send, no replicas, we need replica thread. - sendMultiAction(actionsByServer, numAttempt, (doStartReplica && !hasUnknown) - ? currentActions : null, numAttempt > 1 && !hasUnknown); - } - - if (hasUnknown) { - actionsByServer = new HashMap<>(); - for (Action action : unknownReplicaActions) { - HRegionLocation loc = getReplicaLocationOrFail(action); - if (loc == null) continue; - byte[] regionName = loc.getRegionInfo().getRegionName(); - AsyncProcess.addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup); - } - if (!actionsByServer.isEmpty()) { - sendMultiAction( - actionsByServer, numAttempt, doStartReplica ? currentActions : null, true); - } - } - } - - private HRegionLocation getReplicaLocationOrFail(Action action) { - // We are going to try get location once again. For each action, we'll do it once - // from cache, because the previous calls in the loop might populate it. - int replicaId = action.getReplicaId(); - RegionLocations locs = findAllLocationsOrFail(action, true); - if (locs == null) return null; // manageError already called - HRegionLocation loc = locs.getRegionLocation(replicaId); - if (loc == null || loc.getServerName() == null) { - locs = findAllLocationsOrFail(action, false); - if (locs == null) return null; // manageError already called - loc = locs.getRegionLocation(replicaId); - } - if (loc == null || loc.getServerName() == null) { - manageLocationError(action, null); - return null; - } - return loc; - } - - private void manageLocationError(Action action, Exception ex) { - String msg = "Cannot get replica " + action.getReplicaId() - + " location for " + action.getAction(); - LOG.error(msg); - if (ex == null) { - ex = new IOException(msg); - } - manageError(action.getOriginalIndex(), action.getAction(), - Retry.NO_LOCATION_PROBLEM, ex, null); - } - - private RegionLocations findAllLocationsOrFail(Action action, boolean useCache) { - if (action.getAction() == null) throw new IllegalArgumentException("#" + asyncProcess.id + - ", row cannot be null"); - RegionLocations loc = null; - try { - loc = asyncProcess.connection.locateRegion( - tableName, action.getAction().getRow(), useCache, true, action.getReplicaId()); - } catch (IOException ex) { - manageLocationError(action, ex); - } - return loc; - } - - /** - * Send a multi action structure to the servers, after a delay depending on the attempt - * number. Asynchronous. - * - * @param actionsByServer the actions structured by regions - * @param numAttempt the attempt number. - * @param actionsForReplicaThread original actions for replica thread; null on non-first call. - */ - void sendMultiAction(Map actionsByServer, - int numAttempt, List actionsForReplicaThread, boolean reuseThread) { - // Run the last item on the same thread if we are already on a send thread. - // We hope most of the time it will be the only item, so we can cut down on threads. - int actionsRemaining = actionsByServer.size(); - // This iteration is by server (the HRegionLocation comparator is by server portion only). - for (Map.Entry e : actionsByServer.entrySet()) { - ServerName server = e.getKey(); - MultiAction multiAction = e.getValue(); - Collection runnables = getNewMultiActionRunnable(server, multiAction, - numAttempt); - // make sure we correctly count the number of runnables before we try to reuse the send - // thread, in case we had to split the request into different runnables because of backoff - if (runnables.size() > actionsRemaining) { - actionsRemaining = runnables.size(); - } - - // run all the runnables - // HBASE-17475: Do not reuse the thread after stack reach a certain depth to prevent stack overflow - // for now, we use HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER to control the depth - for (Runnable runnable : runnables) { - if ((--actionsRemaining == 0) && reuseThread - && numAttempt % HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER != 0) { - runnable.run(); - } else { - try { - pool.submit(runnable); - } catch (Throwable t) { - if (t instanceof RejectedExecutionException) { - // This should never happen. But as the pool is provided by the end user, - // let's secure this a little. - LOG.warn("id=" + asyncProcess.id + ", task rejected by pool. Unexpected." + - " Server=" + server.getServerName(), t); - } else { - // see #HBASE-14359 for more details - LOG.warn("Caught unexpected exception/error: ", t); - } - asyncProcess.decTaskCounters(multiAction.getRegions(), server); - // We're likely to fail again, but this will increment the attempt counter, - // so it will finish. - receiveGlobalFailure(multiAction, server, numAttempt, t); - } - } - } - } - - if (actionsForReplicaThread != null) { - startWaitingForReplicaCalls(actionsForReplicaThread); - } - } - - private Collection getNewMultiActionRunnable(ServerName server, - MultiAction multiAction, - int numAttempt) { - // no stats to manage, just do the standard action - if (asyncProcess.connection.getStatisticsTracker() == null) { - if (asyncProcess.connection.getConnectionMetrics() != null) { - asyncProcess.connection.getConnectionMetrics().incrNormalRunners(); - } - asyncProcess.incTaskCounters(multiAction.getRegions(), server); - SingleServerRequestRunnable runnable = createSingleServerRequest( - multiAction, numAttempt, server, callsInProgress); - Tracer tracer = Tracer.curThreadTracer(); - - if (tracer == null) { - return Collections.singletonList(runnable); - } else { - return Collections.singletonList(tracer.wrap(runnable, "AsyncProcess.sendMultiAction")); - } - } - - // group the actions by the amount of delay - Map actions = new HashMap<>(multiAction.size()); - - // split up the actions - for (Map.Entry> e : multiAction.actions.entrySet()) { - Long backoff = getBackoff(server, e.getKey()); - DelayingRunner runner = actions.get(backoff); - if (runner == null) { - actions.put(backoff, new DelayingRunner(backoff, e)); - } else { - runner.add(e); - } - } - - List toReturn = new ArrayList<>(actions.size()); - for (DelayingRunner runner : actions.values()) { - asyncProcess.incTaskCounters(runner.getActions().getRegions(), server); - String traceText = "AsyncProcess.sendMultiAction"; - Runnable runnable = createSingleServerRequest(runner.getActions(), numAttempt, server, callsInProgress); - // use a delay runner only if we need to sleep for some time - if (runner.getSleepTime() > 0) { - runner.setRunner(runnable); - traceText = "AsyncProcess.clientBackoff.sendMultiAction"; - runnable = runner; - if (asyncProcess.connection.getConnectionMetrics() != null) { - asyncProcess.connection.getConnectionMetrics() - .incrDelayRunnersAndUpdateDelayInterval(runner.getSleepTime()); - } - } else { - if (asyncProcess.connection.getConnectionMetrics() != null) { - asyncProcess.connection.getConnectionMetrics().incrNormalRunners(); - } - } - runnable = TraceUtil.wrap(runnable, traceText); - toReturn.add(runnable); - - } - return toReturn; - } - - /** - * @param server server location where the target region is hosted - * @param regionName name of the region which we are going to write some data - * @return the amount of time the client should wait until it submit a request to the - * specified server and region - */ - private Long getBackoff(ServerName server, byte[] regionName) { - ServerStatisticTracker tracker = asyncProcess.connection.getStatisticsTracker(); - ServerStatistics stats = tracker.getStats(server); - return asyncProcess.connection.getBackoffPolicy() - .getBackoffTime(server, regionName, stats); - } - - /** - * Starts waiting to issue replica calls on a different thread; or issues them immediately. - */ - private void startWaitingForReplicaCalls(List actionsForReplicaThread) { - long startTime = EnvironmentEdgeManager.currentTime(); - ReplicaCallIssuingRunnable replicaRunnable = new ReplicaCallIssuingRunnable( - actionsForReplicaThread, startTime); - if (asyncProcess.primaryCallTimeoutMicroseconds == 0) { - // Start replica calls immediately. - replicaRunnable.run(); - } else { - // Start the thread that may kick off replica gets. - // TODO: we could do it on the same thread, but it's a user thread, might be a bad idea. - try { - pool.submit(replicaRunnable); - } catch (RejectedExecutionException ree) { - LOG.warn("id=" + asyncProcess.id + " replica task rejected by pool; no replica calls", ree); - } - } - } - - /** - * Check that we can retry acts accordingly: logs, set the error status. - * - * @param originalIndex the position in the list sent - * @param row the row - * @param canRetry if false, we won't retry whatever the settings. - * @param throwable the throwable, if any (can be null) - * @param server the location, if any (can be null) - * @return true if the action can be retried, false otherwise. - */ - Retry manageError(int originalIndex, Row row, Retry canRetry, - Throwable throwable, ServerName server) { - if (canRetry == Retry.YES - && throwable != null && throwable instanceof DoNotRetryIOException) { - canRetry = Retry.NO_NOT_RETRIABLE; - } - - if (canRetry != Retry.YES) { - // Batch.Callback was not called on failure in 0.94. We keep this. - setError(originalIndex, row, throwable, server); - } else if (isActionComplete(originalIndex, row)) { - canRetry = Retry.NO_OTHER_SUCCEEDED; - } - return canRetry; - } - - /** - * Resubmit all the actions from this multiaction after a failure. - * - * @param rsActions the actions still to do from the initial list - * @param server the destination - * @param numAttempt the number of attempts so far - * @param t the throwable (if any) that caused the resubmit - */ - private void receiveGlobalFailure( - MultiAction rsActions, ServerName server, int numAttempt, Throwable t) { - errorsByServer.reportServerError(server); - Retry canRetry = errorsByServer.canTryMore(numAttempt) - ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED; - - cleanServerCache(server, t); - int failed = 0; - int stopped = 0; - List toReplay = new ArrayList<>(); - for (Map.Entry> e : rsActions.actions.entrySet()) { - byte[] regionName = e.getKey(); - byte[] row = e.getValue().get(0).getAction().getRow(); - // Do not use the exception for updating cache because it might be coming from - // any of the regions in the MultiAction. - updateCachedLocations(server, regionName, row, - ClientExceptionsUtil.isMetaClearingException(t) ? null : t); - for (Action action : e.getValue()) { - Retry retry = manageError( - action.getOriginalIndex(), action.getAction(), canRetry, t, server); - if (retry == Retry.YES) { - toReplay.add(action); - } else if (retry == Retry.NO_OTHER_SUCCEEDED) { - ++stopped; - } else { - ++failed; - } - } - } - - if (toReplay.isEmpty()) { - logNoResubmit(server, numAttempt, rsActions.size(), t, failed, stopped); - } else { - resubmit(server, toReplay, numAttempt, rsActions.size(), t); - } - } - - /** - * Log as much info as possible, and, if there is something to replay, - * submit it again after a back off sleep. - */ - private void resubmit(ServerName oldServer, List toReplay, - int numAttempt, int failureCount, Throwable throwable) { - // We have something to replay. We're going to sleep a little before. - - // We have two contradicting needs here: - // 1) We want to get the new location after having slept, as it may change. - // 2) We want to take into account the location when calculating the sleep time. - // 3) If all this is just because the response needed to be chunked try again FAST. - // It should be possible to have some heuristics to take the right decision. Short term, - // we go for one. - boolean retryImmediately = throwable instanceof RetryImmediatelyException; - int nextAttemptNumber = retryImmediately ? numAttempt : numAttempt + 1; - long backOffTime; - if (retryImmediately) { - backOffTime = 0; - } else if (throwable instanceof CallQueueTooBigException) { - // Give a special check on CQTBE, see #HBASE-17114 - backOffTime = errorsByServer.calculateBackoffTime(oldServer, asyncProcess.pauseForCQTBE); - } else { - backOffTime = errorsByServer.calculateBackoffTime(oldServer, asyncProcess.pause); - } - if (numAttempt > asyncProcess.startLogErrorsCnt) { - // We use this value to have some logs when we have multiple failures, but not too many - // logs, as errors are to be expected when a region moves, splits and so on - LOG.info(createLog(numAttempt, failureCount, toReplay.size(), - oldServer, throwable, backOffTime, true, null, -1, -1)); - } - - try { - if (backOffTime > 0) { - Thread.sleep(backOffTime); - } - } catch (InterruptedException e) { - LOG.warn("#" + asyncProcess.id + ", not sent: " + toReplay.size() + " operations, " + oldServer, e); - Thread.currentThread().interrupt(); - return; - } - - groupAndSendMultiAction(toReplay, nextAttemptNumber); - } - - private void logNoResubmit(ServerName oldServer, int numAttempt, - int failureCount, Throwable throwable, int failed, int stopped) { - if (failureCount != 0 || numAttempt > asyncProcess.startLogErrorsCnt + 1) { - String timeStr = new Date(errorsByServer.getStartTrackingTime()).toString(); - String logMessage = createLog(numAttempt, failureCount, 0, oldServer, - throwable, -1, false, timeStr, failed, stopped); - if (failed != 0) { - // Only log final failures as warning - LOG.warn(logMessage); - } else { - LOG.info(logMessage); - } - } - } - - /** - * Called when we receive the result of a server query. - * - * @param multiAction - the multiAction we sent - * @param server - the location. It's used as a server name. - * @param responses - the response, if any - * @param numAttempt - the attempt - */ - private void receiveMultiAction(MultiAction multiAction, ServerName server, - MultiResponse responses, int numAttempt) { - assert responses != null; - updateStats(server, responses); - // Success or partial success - // Analyze detailed results. We can still have individual failures to be redo. - // two specific throwables are managed: - // - DoNotRetryIOException: we continue to retry for other actions - // - RegionMovedException: we update the cache with the new region location - Map results = responses.getResults(); - List toReplay = new ArrayList<>(); - Throwable lastException = null; - int failureCount = 0; - int failed = 0; - int stopped = 0; - Retry retry = null; - // Go by original action. - for (Map.Entry> regionEntry : multiAction.actions.entrySet()) { - byte[] regionName = regionEntry.getKey(); - - Throwable regionException = responses.getExceptions().get(regionName); - if (regionException != null) { - cleanServerCache(server, regionException); - } - - Map regionResults = - results.containsKey(regionName) ? results.get(regionName).result : Collections.emptyMap(); - boolean regionFailureRegistered = false; - for (Action sentAction : regionEntry.getValue()) { - Object result = regionResults.get(sentAction.getOriginalIndex()); - if (result == null) { - if (regionException == null) { - LOG.error("Server sent us neither results nor exceptions for " - + Bytes.toStringBinary(regionName) - + ", numAttempt:" + numAttempt); - regionException = new RuntimeException("Invalid response"); - } - // If the row operation encounters the region-lever error, the exception of action may be - // null. - result = regionException; - } - // Failure: retry if it's make sense else update the errors lists - if (result instanceof Throwable) { - Throwable actionException = (Throwable) result; - Row row = sentAction.getAction(); - lastException = regionException != null ? regionException - : ClientExceptionsUtil.findException(actionException); - // Register corresponding failures once per server/once per region. - if (!regionFailureRegistered) { - regionFailureRegistered = true; - updateCachedLocations(server, regionName, row.getRow(), actionException); - } - if (retry == null) { - errorsByServer.reportServerError(server); - // We determine canRetry only once for all calls, after reporting server failure. - retry = errorsByServer.canTryMore(numAttempt) ? - Retry.YES : Retry.NO_RETRIES_EXHAUSTED; - } - ++failureCount; - switch (manageError(sentAction.getOriginalIndex(), row, retry, actionException, - server)) { - case YES: - toReplay.add(sentAction); - break; - case NO_OTHER_SUCCEEDED: - ++stopped; - break; - default: - ++failed; - break; - } - } else { - invokeCallBack(regionName, sentAction.getAction().getRow(), (CResult) result); - setResult(sentAction, result); - } - } - } - if (toReplay.isEmpty()) { - logNoResubmit(server, numAttempt, failureCount, lastException, failed, stopped); - } else { - resubmit(server, toReplay, numAttempt, failureCount, lastException); - } - } - - private void updateCachedLocations(ServerName server, byte[] regionName, byte[] row, - Throwable rowException) { - if (tableName == null) { - return; - } - try { - asyncProcess.connection - .updateCachedLocations(tableName, regionName, row, rowException, server); - } catch (Throwable ex) { - // That should never happen, but if it did, we want to make sure - // we still process errors - LOG.error("Couldn't update cached region locations: " + ex); - } - } - - private void invokeCallBack(byte[] regionName, byte[] row, CResult result) { - if (callback != null) { - try { - //noinspection unchecked - // TODO: would callback expect a replica region name if it gets one? - this.callback.update(regionName, row, result); - } catch (Throwable t) { - LOG.error("User callback threw an exception for " - + Bytes.toStringBinary(regionName) + ", ignoring", t); - } - } - } - - private void cleanServerCache(ServerName server, Throwable regionException) { - if (ClientExceptionsUtil.isMetaClearingException(regionException)) { - // We want to make sure to clear the cache in case there were location-related exceptions. - // We don't to clear the cache for every possible exception that comes through, however. - asyncProcess.connection.clearCaches(server); - } - } - - @VisibleForTesting - protected void updateStats(ServerName server, MultiResponse resp) { - ConnectionUtils.updateStats(Optional.ofNullable(asyncProcess.connection.getStatisticsTracker()), - Optional.ofNullable(asyncProcess.connection.getConnectionMetrics()), server, resp); - } - - - private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn, - Throwable error, long backOffTime, boolean willRetry, String startTime, - int failed, int stopped) { - StringBuilder sb = new StringBuilder(); - sb.append("id=").append(asyncProcess.id).append(", table=").append(tableName). - append(", attempt=").append(numAttempt).append("/").append(asyncProcess.numTries). - append(", "); - - if (failureCount > 0 || error != null){ - sb.append("failureCount=").append(failureCount).append("ops").append(", last exception="). - append(error); - } else { - sb.append("succeeded"); - } - - sb.append(" on ").append(sn).append(", tracking started ").append(startTime); - - if (willRetry) { - sb.append(", retrying after=").append(backOffTime).append("ms"). - append(", operationsToReplay=").append(replaySize); - } else if (failureCount > 0) { - if (stopped > 0) { - sb.append("; NOT retrying, stopped=").append(stopped). - append(" because successful operation on other replica"); - } - if (failed > 0) { - sb.append("; NOT retrying, failed=").append(failed).append(" -- final attempt!"); - } - } - - return sb.toString(); - } - - /** - * Sets the non-error result from a particular action. - * @param action Action (request) that the server responded to. - * @param result The result. - */ - private void setResult(Action action, Object result) { - if (result == null) { - throw new RuntimeException("Result cannot be null"); - } - ReplicaResultState state = null; - boolean isStale = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId()); - int index = action.getOriginalIndex(); - if (results == null) { - decActionCounter(index); - return; // Simple case, no replica requests. - } - state = trySetResultSimple(index, action.getAction(), false, result, null, isStale); - if (state == null) { - return; // Simple case, no replica requests. - } - // At this point we know that state is set to replica tracking class. - // It could be that someone else is also looking at it; however, we know there can - // only be one state object, and only one thread can set callCount to 0. Other threads - // will either see state with callCount 0 after locking it; or will not see state at all - // we will replace it with the result. - synchronized (state) { - if (state.callCount == 0) { - return; // someone already set the result - } - state.callCount = 0; - } - synchronized (replicaResultLock) { - if (results[index] != state) { - throw new AssertionError("We set the callCount but someone else replaced the result"); - } - updateResult(index, result); - } - - decActionCounter(index); - } - - /** - * Sets the error from a particular action. - * @param index Original action index. - * @param row Original request. - * @param throwable The resulting error. - * @param server The source server. - */ - private void setError(int index, Row row, Throwable throwable, ServerName server) { - ReplicaResultState state = null; - if (results == null) { - // Note that we currently cannot have replica requests with null results. So it shouldn't - // happen that multiple replica calls will call dAC for same actions with results == null. - // Only one call per action should be present in this case. - errors.add(throwable, row, server); - decActionCounter(index); - return; // Simple case, no replica requests. - } - state = trySetResultSimple(index, row, true, throwable, server, false); - if (state == null) { - return; // Simple case, no replica requests. - } - BatchErrors target = null; // Error will be added to final errors, or temp replica errors. - boolean isActionDone = false; - synchronized (state) { - switch (state.callCount) { - case 0: return; // someone already set the result - case 1: { // All calls failed, we are the last error. - target = errors; - isActionDone = true; - break; - } - default: { - assert state.callCount > 1; - if (state.replicaErrors == null) { - state.replicaErrors = new BatchErrors(); - } - target = state.replicaErrors; - break; - } - } - --state.callCount; - } - target.add(throwable, row, server); - if (isActionDone) { - if (state.replicaErrors != null) { // last call, no need to lock - errors.merge(state.replicaErrors); - } - // See setResult for explanations. - synchronized (replicaResultLock) { - if (results[index] != state) { - throw new AssertionError("We set the callCount but someone else replaced the result"); - } - updateResult(index, throwable); - } - decActionCounter(index); - } - } - - /** - * Checks if the action is complete; used on error to prevent needless retries. - * Does not synchronize, assuming element index/field accesses are atomic. - * This is an opportunistic optimization check, doesn't have to be strict. - * @param index Original action index. - * @param row Original request. - */ - private boolean isActionComplete(int index, Row row) { - if (!AsyncProcess.isReplicaGet(row)) return false; - Object resObj = results[index]; - return (resObj != null) && (!(resObj instanceof ReplicaResultState) - || ((ReplicaResultState)resObj).callCount == 0); - } - - /** - * Tries to set the result or error for a particular action as if there were no replica calls. - * @return null if successful; replica state if there were in fact replica calls. - */ - private ReplicaResultState trySetResultSimple(int index, Row row, boolean isError, - Object result, ServerName server, boolean isFromReplica) { - Object resObj = null; - if (!AsyncProcess.isReplicaGet(row)) { - if (isFromReplica) { - throw new AssertionError("Unexpected stale result for " + row); - } - updateResult(index, result); - } else { - synchronized (replicaResultLock) { - resObj = results[index]; - if (resObj == null) { - if (isFromReplica) { - throw new AssertionError("Unexpected stale result for " + row); - } - updateResult(index, result); - } - } - } - - ReplicaResultState rrs = - (resObj instanceof ReplicaResultState) ? (ReplicaResultState)resObj : null; - if (rrs == null && isError) { - // The resObj is not replica state (null or already set). - errors.add((Throwable)result, row, server); - } - - if (resObj == null) { - // resObj is null - no replica calls were made. - decActionCounter(index); - return null; - } - return rrs; - } - - private void decActionCounter(int index) { - long actionsRemaining = actionsInProgress.decrementAndGet(); - if (actionsRemaining < 0) { - String error = buildDetailedErrorMsg("Incorrect actions in progress", index); - throw new AssertionError(error); - } else if (actionsRemaining == 0) { - synchronized (actionsInProgress) { - actionsInProgress.notifyAll(); - } - } - } - - private String buildDetailedErrorMsg(String string, int index) { - StringBuilder error = new StringBuilder(128); - error.append(string).append("; called for ").append(index).append(", actionsInProgress ") - .append(actionsInProgress.get()).append("; replica gets: "); - if (replicaGetIndices != null) { - for (int i = 0; i < replicaGetIndices.length; ++i) { - error.append(replicaGetIndices[i]).append(", "); - } - } else { - error.append(hasAnyReplicaGets ? "all" : "none"); - } - error.append("; results "); - if (results != null) { - for (int i = 0; i < results.length; ++i) { - Object o = results[i]; - error.append(((o == null) ? "null" : o.toString())).append(", "); - } - } - return error.toString(); - } - - @Override - public void waitUntilDone() throws InterruptedIOException { - try { - waitUntilDone(Long.MAX_VALUE); - } catch (InterruptedException iex) { - throw new InterruptedIOException(iex.getMessage()); - } finally { - if (callsInProgress != null) { - for (CancellableRegionServerCallable clb : callsInProgress) { - clb.cancel(); - } - } - } - } - - private boolean waitUntilDone(long cutoff) throws InterruptedException { - boolean hasWait = cutoff != Long.MAX_VALUE; - long lastLog = EnvironmentEdgeManager.currentTime(); - long currentInProgress; - while (0 != (currentInProgress = actionsInProgress.get())) { - long now = EnvironmentEdgeManager.currentTime(); - if (hasWait && (now * 1000L) > cutoff) { - return false; - } - if (!hasWait) { // Only log if wait is infinite. - if (now > lastLog + 10000) { - lastLog = now; - LOG.info("#" + asyncProcess.id + ", waiting for " + currentInProgress - + " actions to finish on table: " + tableName); - } - } - synchronized (actionsInProgress) { - if (actionsInProgress.get() == 0) break; - if (!hasWait) { - actionsInProgress.wait(10); - } else { - long waitMicroSecond = Math.min(100000L, (cutoff - now * 1000L)); - TimeUnit.MICROSECONDS.timedWait(actionsInProgress, waitMicroSecond); - } - } - } - return true; - } - - @Override - public boolean hasError() { - return errors.hasErrors(); - } - - @Override - public List getFailedOperations() { - return errors.actions; - } - - @Override - public RetriesExhaustedWithDetailsException getErrors() { - return errors.makeException(asyncProcess.logBatchErrorDetails); - } - - @Override - public Object[] getResults() throws InterruptedIOException { - waitUntilDone(); - return results; - } - - /** - * Creates the server error tracker to use inside process. - * Currently, to preserve the main assumption about current retries, and to work well with - * the retry-limit-based calculation, the calculation is local per Process object. - * We may benefit from connection-wide tracking of server errors. - * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection - */ - private ConnectionImplementation.ServerErrorTracker createServerErrorTracker() { - return new ConnectionImplementation.ServerErrorTracker( - asyncProcess.serverTrackerTimeout, asyncProcess.numTries); - } - - /** - * Create a callable. Isolated to be easily overridden in the tests. - */ - private MultiServerCallable createCallable(final ServerName server, TableName tableName, - final MultiAction multi) { - return new MultiServerCallable(asyncProcess.connection, tableName, server, - multi, asyncProcess.rpcFactory.newController(), rpcTimeout, tracker, multi.getPriority()); - } - - private void updateResult(int index, Object result) { - Object current = results[index]; - if (current != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("The result is assigned repeatedly! current:" + current - + ", new:" + result); - } - } - results[index] = result; - } - - @VisibleForTesting - long getNumberOfActionsInProgress() { - return actionsInProgress.get(); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BatchErrors.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BatchErrors.java deleted file mode 100644 index d3cdc74fdf60..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BatchErrors.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * - * 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.hbase.ServerName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; - -class BatchErrors { - private static final Logger LOG = LoggerFactory.getLogger(BatchErrors.class); - final List throwables = new ArrayList<>(); - final List actions = new ArrayList<>(); - final List addresses = new ArrayList<>(); - - public synchronized void add(Throwable ex, Row row, ServerName serverName) { - if (row == null){ - throw new IllegalArgumentException("row cannot be null. location=" + serverName); - } - - throwables.add(ex); - actions.add(row); - addresses.add(serverName != null ? serverName.toString() : "null"); - } - - public boolean hasErrors() { - return !throwables.isEmpty(); - } - - synchronized RetriesExhaustedWithDetailsException makeException(boolean logDetails) { - if (logDetails) { - LOG.error("Exception occurred! Exception details: " + throwables + ";\nActions: " - + actions); - } - return new RetriesExhaustedWithDetailsException(new ArrayList<>(throwables), - new ArrayList<>(actions), new ArrayList<>(addresses)); - } - - public synchronized void clear() { - throwables.clear(); - actions.clear(); - addresses.clear(); - } - - public synchronized void merge(BatchErrors other) { - throwables.addAll(other.throwables); - actions.addAll(other.actions); - addresses.addAll(other.addresses); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java deleted file mode 100644 index 6d70219b73fe..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ /dev/null @@ -1,500 +0,0 @@ -/** - * - * 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 static org.apache.hadoop.hbase.client.BufferedMutatorParams.UNSET; - -import java.io.Closeable; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.ConcurrentLinkedQueue; -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.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -/** - *

- * Used to communicate with a single HBase table similar to {@link Table} - * but meant for batched, potentially asynchronous puts. Obtain an instance from - * a {@link Connection} and call {@link #close()} afterwards. Provide an alternate - * to this implementation by setting {@link BufferedMutatorParams#implementationClassName(String)} - * or by setting alternate classname via the key {} in Configuration. - *

- * - *

- * While this can be used across threads, great care should be used when doing so. - * Errors are global to the buffered mutator and the Exceptions can be thrown on any - * thread that causes the flush for requests. - *

- * - * @see ConnectionFactory - * @see Connection - * @since 1.0.0 - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -class BufferedMutatorImpl implements BufferedMutator { - - private static final Logger LOG = LoggerFactory.getLogger(BufferedMutatorImpl.class); - - private final ExceptionListener listener; - - private final TableName tableName; - - 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. - */ - private final AtomicInteger undealtMutationCount = new AtomicInteger(0); - private final long writeBufferSize; - - private final AtomicLong writeBufferPeriodicFlushTimeoutMs = new AtomicLong(0); - private final AtomicLong writeBufferPeriodicFlushTimerTickMs = - new AtomicLong(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); - private Timer writeBufferPeriodicFlushTimer = null; - - private final int maxKeyValueSize; - private final ExecutorService pool; - private final AtomicInteger rpcTimeout; - private final AtomicInteger operationTimeout; - private final boolean cleanupPoolOnClose; - private volatile boolean closed = false; - private final AsyncProcess ap; - - @VisibleForTesting - BufferedMutatorImpl(ConnectionImplementation conn, BufferedMutatorParams params, - AsyncProcess ap) { - if (conn == null || conn.isClosed()) { - throw new IllegalArgumentException("Connection is null or closed."); - } - this.tableName = params.getTableName(); - 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() != UNSET ? - params.getWriteBufferSize() : tableConf.getWriteBufferSize(); - - // Set via the setter because it does value validation and starts/stops the TimerTask - long newWriteBufferPeriodicFlushTimeoutMs = - params.getWriteBufferPeriodicFlushTimeoutMs() != UNSET - ? params.getWriteBufferPeriodicFlushTimeoutMs() - : tableConf.getWriteBufferPeriodicFlushTimeoutMs(); - long newWriteBufferPeriodicFlushTimerTickMs = - params.getWriteBufferPeriodicFlushTimerTickMs() != UNSET - ? params.getWriteBufferPeriodicFlushTimerTickMs() - : tableConf.getWriteBufferPeriodicFlushTimerTickMs(); - this.setWriteBufferPeriodicFlush( - newWriteBufferPeriodicFlushTimeoutMs, - newWriteBufferPeriodicFlushTimerTickMs); - - this.maxKeyValueSize = - params.getMaxKeyValueSize() != UNSET ? - params.getMaxKeyValueSize() : tableConf.getMaxKeyValueSize(); - - this.rpcTimeout = new AtomicInteger( - params.getRpcTimeout() != UNSET ? - params.getRpcTimeout() : conn.getConnectionConfiguration().getWriteRpcTimeout()); - - this.operationTimeout = new AtomicInteger( - params.getOperationTimeout() != UNSET ? - params.getOperationTimeout() : conn.getConnectionConfiguration().getOperationTimeout()); - this.ap = ap; - } - - BufferedMutatorImpl(ConnectionImplementation 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, rpcFactory)); - } - - private void checkClose() { - if (closed) { - throw new IllegalStateException("Cannot put when the BufferedMutator is closed."); - } - } - - @VisibleForTesting - ExecutorService getPool() { - return pool; - } - - @VisibleForTesting - AsyncProcess getAsyncProcess() { - return ap; - } - - @Override - public TableName getName() { - return tableName; - } - - @Override - public Configuration getConfiguration() { - return conf; - } - - @Override - public void mutate(Mutation m) throws InterruptedIOException, - RetriesExhaustedWithDetailsException { - mutate(Collections.singletonList(m)); - } - - @Override - public void mutate(List ms) throws InterruptedIOException, - RetriesExhaustedWithDetailsException { - checkClose(); - - long toAddSize = 0; - int toAddCount = 0; - for (Mutation m : ms) { - if (m instanceof Put) { - ConnectionUtils.validatePut((Put) m, maxKeyValueSize); - } - toAddSize += m.heapSize(); - ++toAddCount; - } - - if (currentWriteBufferSize.get() == 0) { - firstRecordInBufferTimestamp.set(System.currentTimeMillis()); - } - currentWriteBufferSize.addAndGet(toAddSize); - writeAsyncBuffer.addAll(ms); - undealtMutationCount.addAndGet(toAddCount); - doFlush(false); - } - - @VisibleForTesting - protected long getExecutedWriteBufferPeriodicFlushes() { - return executedWriteBufferPeriodicFlushes.get(); - } - - private final AtomicLong firstRecordInBufferTimestamp = new AtomicLong(0); - private final AtomicLong executedWriteBufferPeriodicFlushes = new AtomicLong(0); - - private void timerCallbackForWriteBufferPeriodicFlush() { - if (currentWriteBufferSize.get() == 0) { - return; // Nothing to flush - } - long now = System.currentTimeMillis(); - if (firstRecordInBufferTimestamp.get() + writeBufferPeriodicFlushTimeoutMs.get() > now) { - return; // No need to flush yet - } - // The first record in the writebuffer has been in there too long --> flush - try { - executedWriteBufferPeriodicFlushes.incrementAndGet(); - flush(); - } catch (InterruptedIOException | RetriesExhaustedWithDetailsException e) { - LOG.error("Exception during timerCallbackForWriteBufferPeriodicFlush --> " + e.getMessage()); - } - } - - @Override - public synchronized void close() throws IOException { - if (closed) { - return; - } - // Stop any running Periodic Flush timer. - disableWriteBufferPeriodicFlush(); - try { - // As we can have an operation in progress even if the buffer is empty, we call - // doFlush at least one time. - doFlush(true); - } finally { - if (cleanupPoolOnClose) { - this.pool.shutdown(); - try { - if (!pool.awaitTermination(600, TimeUnit.SECONDS)) { - LOG.warn("close() failed to terminate pool after 10 minutes. Abandoning pool."); - } - } catch (InterruptedException e) { - LOG.warn("waitForTermination interrupted"); - Thread.currentThread().interrupt(); - } - } - closed = true; - } - } - - private AsyncProcessTask createTask(QueueRowAccess access) { - return new AsyncProcessTask(AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(access) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.AT_LEAST_ONE) - .build()) { - @Override - public int getRpcTimeout() { - return rpcTimeout.get(); - } - - @Override - public int getOperationTimeout() { - return operationTimeout.get(); - } - }; - } - - @Override - public void flush() throws InterruptedIOException, RetriesExhaustedWithDetailsException { - checkClose(); - doFlush(true); - } - - /** - * Send the operations in the buffer to the servers. - * - * @param flushAll - if true, sends all the writes and wait for all of them to finish before - * returning. Otherwise, flush until buffer size is smaller than threshold - */ - private void doFlush(boolean flushAll) throws InterruptedIOException, - RetriesExhaustedWithDetailsException { - List errors = new ArrayList<>(); - while (true) { - if (!flushAll && currentWriteBufferSize.get() <= writeBufferSize) { - // There is the room to accept more mutations. - break; - } - AsyncRequestFuture asf; - try (QueueRowAccess access = createQueueRowAccess()) { - if (access.isEmpty()) { - // It means someone has gotten the ticker to run the flush. - break; - } - asf = ap.submit(createTask(access)); - } - // DON'T do the wait in the try-with-resources. Otherwise, the undealt mutations won't - // be released. - asf.waitUntilDone(); - if (asf.hasError()) { - errors.add(asf.getErrors()); - } - } - - RetriesExhaustedWithDetailsException exception = makeException(errors); - if (exception == null) { - return; - } else if(listener == null) { - throw exception; - } else { - listener.onException(exception, this); - } - } - - private static RetriesExhaustedWithDetailsException makeException( - List errors) { - switch (errors.size()) { - case 0: - return null; - case 1: - return errors.get(0); - default: - List exceptions = new ArrayList<>(); - List actions = new ArrayList<>(); - List hostnameAndPort = new ArrayList<>(); - errors.forEach(e -> { - exceptions.addAll(e.exceptions); - actions.addAll(e.actions); - hostnameAndPort.addAll(e.hostnameAndPort); - }); - return new RetriesExhaustedWithDetailsException(exceptions, actions, hostnameAndPort); - } - } - - /** - * {@inheritDoc} - */ - @Override - public long getWriteBufferSize() { - return this.writeBufferSize; - } - - @Override - public synchronized void setWriteBufferPeriodicFlush(long timeoutMs, long timerTickMs) { - long originalTimeoutMs = this.writeBufferPeriodicFlushTimeoutMs.get(); - long originalTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs.get(); - - // Both parameters have minimal values. - writeBufferPeriodicFlushTimeoutMs.set(Math.max(0, timeoutMs)); - writeBufferPeriodicFlushTimerTickMs.set( - Math.max(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS, timerTickMs)); - - // If something changed we stop the old Timer. - if (writeBufferPeriodicFlushTimeoutMs.get() != originalTimeoutMs || - writeBufferPeriodicFlushTimerTickMs.get() != originalTimerTickMs) { - if (writeBufferPeriodicFlushTimer != null) { - writeBufferPeriodicFlushTimer.cancel(); - writeBufferPeriodicFlushTimer = null; - } - } - - // If we have the need for a timer and there is none we start it - if (writeBufferPeriodicFlushTimer == null && - writeBufferPeriodicFlushTimeoutMs.get() > 0) { - writeBufferPeriodicFlushTimer = new Timer(true); // Create Timer running as Daemon. - writeBufferPeriodicFlushTimer.schedule(new TimerTask() { - @Override - public void run() { - BufferedMutatorImpl.this.timerCallbackForWriteBufferPeriodicFlush(); - } - }, writeBufferPeriodicFlushTimerTickMs.get(), - writeBufferPeriodicFlushTimerTickMs.get()); - } - } - - @Override - public long getWriteBufferPeriodicFlushTimeoutMs() { - return writeBufferPeriodicFlushTimeoutMs.get(); - } - - @Override - public long getWriteBufferPeriodicFlushTimerTickMs() { - return writeBufferPeriodicFlushTimerTickMs.get(); - } - - @Override - public void setRpcTimeout(int rpcTimeout) { - this.rpcTimeout.set(rpcTimeout); - } - - @Override - public void setOperationTimeout(int operationTimeout) { - this.operationTimeout.set(operationTimeout); - } - - @VisibleForTesting - long getCurrentWriteBufferSize() { - return currentWriteBufferSize.get(); - } - - /** - * Count the mutations which haven't been processed. - * @return count of undealt mutation - */ - @VisibleForTesting - int size() { - return undealtMutationCount.get(); - } - - /** - * Count the mutations which haven't been flushed - * @return count of unflushed mutation - */ - @VisibleForTesting - int getUnflushedSize() { - return writeAsyncBuffer.size(); - } - - @VisibleForTesting - QueueRowAccess createQueueRowAccess() { - return new QueueRowAccess(); - } - - @VisibleForTesting - class QueueRowAccess implements RowAccess, Closeable { - private int remainder = undealtMutationCount.getAndSet(0); - private Mutation last = null; - - private void restoreLastMutation() { - // restore the last mutation since it isn't submitted - if (last != null) { - writeAsyncBuffer.add(last); - currentWriteBufferSize.addAndGet(last.heapSize()); - last = null; - } - } - - @Override - public void close() { - restoreLastMutation(); - if (remainder > 0) { - undealtMutationCount.addAndGet(remainder); - remainder = 0; - } - } - - @Override - public Iterator iterator() { - return new Iterator() { - private int countDown = remainder; - @Override - public boolean hasNext() { - return countDown > 0; - } - @Override - public Row next() { - restoreLastMutation(); - if (!hasNext()) { - throw new NoSuchElementException(); - } - last = writeAsyncBuffer.poll(); - if (last == null) { - throw new NoSuchElementException(); - } - currentWriteBufferSize.addAndGet(-last.heapSize()); - --countDown; - return last; - } - @Override - public void remove() { - if (last == null) { - throw new IllegalStateException(); - } - --remainder; - last = null; - } - }; - } - - @Override - public int size() { - return remainder; - } - - @Override - public boolean isEmpty() { - return remainder <= 0; - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cancellable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cancellable.java deleted file mode 100644 index 5095c96ab327..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cancellable.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * - * 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.yetus.audience.InterfaceAudience; - -/** - * This should be implemented by the Get/Scan implementations that - * talk to replica regions. When an RPC response is received from one - * of the replicas, the RPCs to the other replicas are cancelled. - */ -@InterfaceAudience.Private -interface Cancellable { - public void cancel(); - public boolean isCancelled(); -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java deleted file mode 100644 index f81018ea1584..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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 org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; - -/** - * This class is used to unify HTable calls with AsyncProcess Framework. HTable can use - * AsyncProcess directly though this class. Also adds global timeout tracking on top of - * RegionServerCallable and implements Cancellable. - * Global timeout tracking conflicts with logic in RpcRetryingCallerImpl's callWithRetries. So you - * can only use this callable in AsyncProcess which only uses callWithoutRetries and retries in its - * own implementation. - */ -@InterfaceAudience.Private -abstract class CancellableRegionServerCallable extends ClientServiceCallable implements - Cancellable { - private final RetryingTimeTracker tracker; - private final int rpcTimeout; - - CancellableRegionServerCallable(ConnectionImplementation connection, TableName tableName, - byte[] row, RpcController rpcController, int rpcTimeout, RetryingTimeTracker tracker, - int priority) { - super(connection, tableName, row, rpcController, priority); - this.rpcTimeout = rpcTimeout; - this.tracker = tracker; - } - - /* Override so can mess with the callTimeout. - * (non-Javadoc) - * @see org.apache.hadoop.hbase.client.RegionServerCallable#rpcCall(int) - */ - @Override - public T call(int operationTimeout) throws IOException { - if (isCancelled()) return null; - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - // It is expected (it seems) that tracker.start can be called multiple times (on each trip - // through the call when retrying). Also, we can call start and no need of a stop. - this.tracker.start(); - int remainingTime = tracker.getRemainingTime(operationTimeout); - if (remainingTime <= 1) { - // "1" is a special return value in RetryingTimeTracker, see its implementation. - throw new DoNotRetryIOException("Operation rpcTimeout"); - } - return super.call(Math.min(rpcTimeout, remainingTime)); - } - - @Override - public void prepare(boolean reload) throws IOException { - if (isCancelled()) return; - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - super.prepare(reload); - } - - @Override - protected void setStubByServiceName(ServerName serviceName) throws IOException { - setStub(getConnection().getClient(serviceName)); - } - - @Override - public void cancel() { - getRpcController().startCancel(); - } - - @Override - public boolean isCancelled() { - return getRpcController().isCanceled(); - } - - protected ClientProtos.MultiResponse doMulti(ClientProtos.MultiRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().multi(getRpcController(), request); - } - - protected ClientProtos.ScanResponse doScan(ClientProtos.ScanRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().scan(getRpcController(), request); - } - - protected ClientProtos.PrepareBulkLoadResponse doPrepareBulkLoad( - ClientProtos.PrepareBulkLoadRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().prepareBulkLoad(getRpcController(), request); - } - - protected ClientProtos.BulkLoadHFileResponse doBulkLoadHFile( - ClientProtos.BulkLoadHFileRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().bulkLoadHFile(getRpcController(), request); - } - - protected ClientProtos.CleanupBulkLoadResponse doCleanupBulkLoad( - ClientProtos.CleanupBulkLoadRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().cleanupBulkLoad(getRpcController(), request); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java deleted file mode 100644 index 76d74096306e..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java +++ /dev/null @@ -1,193 +0,0 @@ -/** - * 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.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Consumer; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Threads; - -/** - * ClientAsyncPrefetchScanner implements async scanner behaviour. - * Specifically, the cache used by this scanner is a concurrent queue which allows both - * the producer (hbase client) and consumer (application) to access the queue in parallel. - * The number of rows returned in a prefetch is defined by the caching factor and the result size - * factor. - * This class allocates a buffer cache, whose size is a function of both factors. - * The prefetch is invoked when the cache is half­filled, instead of waiting for it to be empty. - * This is defined in the method {@link ClientAsyncPrefetchScanner#prefetchCondition()}. - */ -@InterfaceAudience.Private -public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { - - private long maxCacheSize; - private AtomicLong cacheSizeInBytes; - // exception queue (from prefetch to main scan execution) - private Queue exceptionsQueue; - // prefetch thread to be executed asynchronously - private Thread prefetcher; - // used for testing - private Consumer prefetchListener; - - private final Lock lock = new ReentrantLock(); - private final Condition notEmpty = lock.newCondition(); - private final Condition notFull = lock.newCondition(); - - public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name, - ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory, - RpcControllerFactory rpcControllerFactory, ExecutorService pool, - int replicaCallTimeoutMicroSecondScan) throws IOException { - super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, - replicaCallTimeoutMicroSecondScan); - } - - @VisibleForTesting - void setPrefetchListener(Consumer prefetchListener) { - this.prefetchListener = prefetchListener; - } - - @Override - protected void initCache() { - // concurrent cache - maxCacheSize = resultSize2CacheSize(maxScannerResultSize); - cache = new LinkedBlockingQueue<>(); - cacheSizeInBytes = new AtomicLong(0); - exceptionsQueue = new ConcurrentLinkedQueue<>(); - prefetcher = new Thread(new PrefetchRunnable()); - Threads.setDaemonThreadRunning(prefetcher, tableName + ".asyncPrefetcher"); - } - - private long resultSize2CacheSize(long maxResultSize) { - // * 2 if possible - return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2; - } - - @Override - public Result next() throws IOException { - try { - lock.lock(); - while (cache.isEmpty()) { - handleException(); - if (this.closed) { - return null; - } - try { - notEmpty.await(); - } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted when wait to load cache"); - } - } - - Result result = pollCache(); - if (prefetchCondition()) { - notFull.signalAll(); - } - return result; - } finally { - lock.unlock(); - handleException(); - } - } - - @Override - public void close() { - try { - lock.lock(); - super.close(); - closed = true; - notFull.signalAll(); - notEmpty.signalAll(); - } finally { - lock.unlock(); - } - } - - @Override - protected void addEstimatedSize(long estimatedSize) { - cacheSizeInBytes.addAndGet(estimatedSize); - } - - private void handleException() throws IOException { - //The prefetch task running in the background puts any exception it - //catches into this exception queue. - // Rethrow the exception so the application can handle it. - while (!exceptionsQueue.isEmpty()) { - Exception first = exceptionsQueue.peek(); - if (first instanceof IOException) { - throw (IOException) first; - } - throw (RuntimeException) first; - } - } - - private boolean prefetchCondition() { - return cacheSizeInBytes.get() < maxCacheSize / 2; - } - - private Result pollCache() { - Result res = cache.poll(); - long estimatedSize = calcEstimatedSize(res); - addEstimatedSize(-estimatedSize); - return res; - } - - private class PrefetchRunnable implements Runnable { - - @Override - public void run() { - while (!closed) { - boolean succeed = false; - try { - lock.lock(); - while (!prefetchCondition()) { - notFull.await(); - } - loadCache(); - succeed = true; - } catch (Exception e) { - exceptionsQueue.add(e); - } finally { - notEmpty.signalAll(); - lock.unlock(); - if (prefetchListener != null) { - prefetchListener.accept(succeed); - } - } - } - } - - } - -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java deleted file mode 100644 index fb899255f9a6..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java +++ /dev/null @@ -1,566 +0,0 @@ -/** - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize; -import static org.apache.hadoop.hbase.client.ConnectionUtils.createScanResultCache; -import static org.apache.hadoop.hbase.client.ConnectionUtils.incRegionCountMetrics; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayDeque; -import java.util.Queue; -import java.util.concurrent.ExecutorService; -import org.apache.commons.lang3.mutable.MutableBoolean; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.NotServingRegionException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; -import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; -import org.apache.hadoop.hbase.exceptions.ScannerResetException; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.regionserver.LeaseException; -import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -/** - * Implements the scanner interface for the HBase client. If there are multiple regions in a table, - * this scanner will iterate through them all. - */ -@InterfaceAudience.Private -public abstract class ClientScanner extends AbstractClientScanner { - - private static final Logger LOG = LoggerFactory.getLogger(ClientScanner.class); - - protected final Scan scan; - protected boolean closed = false; - // Current region scanner is against. Gets cleared if current region goes - // wonky: e.g. if it splits on us. - protected HRegionInfo currentRegion = null; - protected ScannerCallableWithReplicas callable = null; - protected Queue cache; - private final ScanResultCache scanResultCache; - protected final int caching; - protected long lastNext; - // Keep lastResult returned successfully in case we have to reset scanner. - protected Result lastResult = null; - protected final long maxScannerResultSize; - private final ConnectionImplementation connection; - protected final TableName tableName; - protected final int scannerTimeout; - protected RpcRetryingCaller caller; - protected RpcControllerFactory rpcControllerFactory; - protected Configuration conf; - // The timeout on the primary. Applicable if there are multiple replicas for a region - // In that case, we will only wait for this much timeout on the primary before going - // to the replicas and trying the same scan. Note that the retries will still happen - // on each replica and the first successful results will be taken. A timeout of 0 is - // disallowed. - protected final int primaryOperationTimeout; - private int retries; - protected final ExecutorService pool; - - /** - * Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start - * row maybe changed changed. - * @param conf The {@link Configuration} to use. - * @param scan {@link Scan} to use in this scanner - * @param tableName The table that we wish to scan - * @param connection Connection identifying the cluster - * @throws IOException - */ - public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName, - ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) - throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace( - "Scan table=" + tableName + ", startRow=" + Bytes.toStringBinary(scan.getStartRow())); - } - this.scan = scan; - this.tableName = tableName; - this.lastNext = System.currentTimeMillis(); - this.connection = connection; - this.pool = pool; - this.primaryOperationTimeout = primaryOperationTimeout; - this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, - HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); - if (scan.getMaxResultSize() > 0) { - this.maxScannerResultSize = scan.getMaxResultSize(); - } else { - this.maxScannerResultSize = conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, - HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); - } - this.scannerTimeout = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, - HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD); - - // check if application wants to collect scan metrics - initScanMetrics(scan); - - // Use the caching from the Scan. If not set, use the default cache setting for this table. - if (this.scan.getCaching() > 0) { - this.caching = this.scan.getCaching(); - } else { - this.caching = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, - HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING); - } - - this.caller = rpcFactory. newCaller(); - this.rpcControllerFactory = controllerFactory; - - this.conf = conf; - - this.scanResultCache = createScanResultCache(scan); - initCache(); - } - - protected ConnectionImplementation getConnection() { - return this.connection; - } - - protected TableName getTable() { - return this.tableName; - } - - protected int getRetries() { - return this.retries; - } - - protected int getScannerTimeout() { - return this.scannerTimeout; - } - - protected Configuration getConf() { - return this.conf; - } - - protected Scan getScan() { - return scan; - } - - protected ExecutorService getPool() { - return pool; - } - - protected int getPrimaryOperationTimeout() { - return primaryOperationTimeout; - } - - protected int getCaching() { - return caching; - } - - protected long getTimestamp() { - return lastNext; - } - - @VisibleForTesting - protected long getMaxResultSize() { - return maxScannerResultSize; - } - - private void closeScanner() throws IOException { - if (this.callable != null) { - this.callable.setClose(); - call(callable, caller, scannerTimeout, false); - this.callable = null; - } - } - - /** - * Will be called in moveToNextRegion when currentRegion is null. Abstract because for normal - * scan, we will start next scan from the endKey of the currentRegion, and for reversed scan, we - * will start next scan from the startKey of the currentRegion. - * @return {@code false} if we have reached the stop row. Otherwise {@code true}. - */ - protected abstract boolean setNewStartKey(); - - /** - * Will be called in moveToNextRegion to create ScannerCallable. Abstract because for reversed - * scan we need to create a ReversedScannerCallable. - */ - protected abstract ScannerCallable createScannerCallable(); - - /** - * Close the previous scanner and create a new ScannerCallable for the next scanner. - *

- * Marked as protected only because TestClientScanner need to override this method. - * @return false if we should terminate the scan. Otherwise - */ - @VisibleForTesting - protected boolean moveToNextRegion() { - // Close the previous scanner if it's open - try { - closeScanner(); - } catch (IOException e) { - // not a big deal continue - if (LOG.isDebugEnabled()) { - LOG.debug("close scanner for " + currentRegion + " failed", e); - } - } - if (currentRegion != null) { - if (!setNewStartKey()) { - return false; - } - scan.resetMvccReadPoint(); - if (LOG.isTraceEnabled()) { - LOG.trace("Finished " + this.currentRegion); - } - } - if (LOG.isDebugEnabled() && this.currentRegion != null) { - // Only worth logging if NOT first region in scan. - LOG.debug( - "Advancing internal scanner to startKey at '" + Bytes.toStringBinary(scan.getStartRow()) + - "', " + (scan.includeStartRow() ? "inclusive" : "exclusive")); - } - // clear the current region, we will set a new value to it after the first call of the new - // callable. - this.currentRegion = null; - this.callable = - new ScannerCallableWithReplicas(getTable(), getConnection(), createScannerCallable(), pool, - primaryOperationTimeout, scan, getRetries(), scannerTimeout, caching, conf, caller); - this.callable.setCaching(this.caching); - incRegionCountMetrics(scanMetrics); - return true; - } - - @VisibleForTesting - boolean isAnyRPCcancelled() { - return callable.isAnyRPCcancelled(); - } - - private Result[] call(ScannerCallableWithReplicas callable, RpcRetryingCaller caller, - int scannerTimeout, boolean updateCurrentRegion) throws IOException { - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - // callWithoutRetries is at this layer. Within the ScannerCallableWithReplicas, - // we do a callWithRetries - Result[] rrs = caller.callWithoutRetries(callable, scannerTimeout); - if (currentRegion == null && updateCurrentRegion) { - currentRegion = callable.getHRegionInfo(); - } - return rrs; - } - - protected void initSyncCache() { - cache = new ArrayDeque<>(); - } - - protected Result nextWithSyncCache() throws IOException { - Result result = cache.poll(); - if (result != null) { - return result; - } - // If there is nothing left in the cache and the scanner is closed, - // return a no-op - if (this.closed) { - return null; - } - - loadCache(); - - // try again to load from cache - result = cache.poll(); - return result; - } - - @VisibleForTesting - public int getCacheSize() { - return cache != null ? cache.size() : 0; - } - - private boolean scanExhausted(Result[] values) { - return callable.moreResultsForScan() == MoreResults.NO; - } - - private boolean regionExhausted(Result[] values) { - // 1. Not a heartbeat message and we get nothing, this means the region is exhausted. And in the - // old time we always return empty result for a open scanner operation so we add a check here to - // keep compatible with the old logic. Should remove the isOpenScanner in the future. - // 2. Server tells us that it has no more results for this region. - return (values.length == 0 && !callable.isHeartbeatMessage()) || - callable.moreResultsInRegion() == MoreResults.NO; - } - - private void closeScannerIfExhausted(boolean exhausted) throws IOException { - if (exhausted) { - closeScanner(); - } - } - - private void handleScanError(DoNotRetryIOException e, - MutableBoolean retryAfterOutOfOrderException, int retriesLeft) throws DoNotRetryIOException { - // An exception was thrown which makes any partial results that we were collecting - // invalid. The scanner will need to be reset to the beginning of a row. - scanResultCache.clear(); - - // Unfortunately, DNRIOE is used in two different semantics. - // (1) The first is to close the client scanner and bubble up the exception all the way - // to the application. This is preferred when the exception is really un-recoverable - // (like CorruptHFileException, etc). Plain DoNotRetryIOException also falls into this - // bucket usually. - // (2) Second semantics is to close the current region scanner only, but continue the - // client scanner by overriding the exception. This is usually UnknownScannerException, - // OutOfOrderScannerNextException, etc where the region scanner has to be closed, but the - // application-level ClientScanner has to continue without bubbling up the exception to - // the client. See RSRpcServices to see how it throws DNRIOE's. - // See also: HBASE-16604, HBASE-17187 - - // If exception is any but the list below throw it back to the client; else setup - // the scanner and retry. - Throwable cause = e.getCause(); - if ((cause != null && cause instanceof NotServingRegionException) || - (cause != null && cause instanceof RegionServerStoppedException) || - e instanceof OutOfOrderScannerNextException || e instanceof UnknownScannerException || - e instanceof ScannerResetException || e instanceof LeaseException) { - // Pass. It is easier writing the if loop test as list of what is allowed rather than - // as a list of what is not allowed... so if in here, it means we do not throw. - if (retriesLeft <= 0) { - throw e; // no more retries - } - } else { - throw e; - } - - // Else, its signal from depths of ScannerCallable that we need to reset the scanner. - if (this.lastResult != null) { - // The region has moved. We need to open a brand new scanner at the new location. - // Reset the startRow to the row we've seen last so that the new scanner starts at - // the correct row. Otherwise we may see previously returned rows again. - // If the lastRow is not partial, then we should start from the next row. As now we can - // exclude the start row, the logic here is the same for both normal scan and reversed scan. - // If lastResult is partial then include it, otherwise exclude it. - scan.withStartRow(lastResult.getRow(), lastResult.mayHaveMoreCellsInRow()); - } - if (e instanceof OutOfOrderScannerNextException) { - if (retryAfterOutOfOrderException.isTrue()) { - retryAfterOutOfOrderException.setValue(false); - } else { - // TODO: Why wrap this in a DNRIOE when it already is a DNRIOE? - throw new DoNotRetryIOException( - "Failed after retry of OutOfOrderScannerNextException: was there a rpc timeout?", e); - } - } - // Clear region. - this.currentRegion = null; - // Set this to zero so we don't try and do an rpc and close on remote server when - // the exception we got was UnknownScanner or the Server is going down. - callable = null; - } - - /** - * Contact the servers to load more {@link Result}s in the cache. - */ - protected void loadCache() throws IOException { - // check if scanner was closed during previous prefetch - if (closed) { - return; - } - long remainingResultSize = maxScannerResultSize; - int countdown = this.caching; - // This is possible if we just stopped at the boundary of a region in the previous call. - if (callable == null && !moveToNextRegion()) { - closed = true; - return; - } - // This flag is set when we want to skip the result returned. We do - // this when we reset scanner because it split under us. - MutableBoolean retryAfterOutOfOrderException = new MutableBoolean(true); - // Even if we are retrying due to UnknownScannerException, ScannerResetException, etc. we should - // make sure that we are not retrying indefinitely. - int retriesLeft = getRetries(); - for (;;) { - Result[] values; - try { - // Server returns a null values if scanning is to stop. Else, - // returns an empty array if scanning is to go on and we've just - // exhausted current region. - // now we will also fetch data when openScanner, so do not make a next call again if values - // is already non-null. - values = call(callable, caller, scannerTimeout, true); - // When the replica switch happens, we need to do certain operations again. - // The callable will openScanner with the right startkey but we need to pick up - // from there. Bypass the rest of the loop and let the catch-up happen in the beginning - // of the loop as it happens for the cases where we see exceptions. - if (callable.switchedToADifferentReplica()) { - // Any accumulated partial results are no longer valid since the callable will - // openScanner with the correct startkey and we must pick up from there - scanResultCache.clear(); - this.currentRegion = callable.getHRegionInfo(); - } - retryAfterOutOfOrderException.setValue(true); - } catch (DoNotRetryIOException e) { - handleScanError(e, retryAfterOutOfOrderException, retriesLeft--); - // reopen the scanner - if (!moveToNextRegion()) { - break; - } - continue; - } - long currentTime = System.currentTimeMillis(); - if (this.scanMetrics != null) { - this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext); - } - lastNext = currentTime; - // Groom the array of Results that we received back from the server before adding that - // Results to the scanner's cache. If partial results are not allowed to be seen by the - // caller, all book keeping will be performed within this method. - int numberOfCompleteRowsBefore = scanResultCache.numberOfCompleteRows(); - Result[] resultsToAddToCache = - scanResultCache.addAndGet(values, callable.isHeartbeatMessage()); - int numberOfCompleteRows = - scanResultCache.numberOfCompleteRows() - numberOfCompleteRowsBefore; - for (Result rs : resultsToAddToCache) { - cache.add(rs); - long estimatedHeapSizeOfResult = calcEstimatedSize(rs); - countdown--; - remainingResultSize -= estimatedHeapSizeOfResult; - addEstimatedSize(estimatedHeapSizeOfResult); - this.lastResult = rs; - } - - if (scan.getLimit() > 0) { - int newLimit = scan.getLimit() - numberOfCompleteRows; - assert newLimit >= 0; - scan.setLimit(newLimit); - } - if (scan.getLimit() == 0 || scanExhausted(values)) { - closeScanner(); - closed = true; - break; - } - boolean regionExhausted = regionExhausted(values); - if (callable.isHeartbeatMessage()) { - if (!cache.isEmpty()) { - // Caller of this method just wants a Result. If we see a heartbeat message, it means - // processing of the scan is taking a long time server side. Rather than continue to - // loop until a limit (e.g. size or caching) is reached, break out early to avoid causing - // unnecesary delays to the caller - LOG.trace("Heartbeat message received and cache contains Results. " + - "Breaking out of scan loop"); - // we know that the region has not been exhausted yet so just break without calling - // closeScannerIfExhausted - break; - } - } - if (cache.isEmpty() && !closed && scan.isNeedCursorResult()) { - if (callable.isHeartbeatMessage() && callable.getCursor() != null) { - // Use cursor row key from server - cache.add(Result.createCursorResult(callable.getCursor())); - break; - } - if (values.length > 0) { - // It is size limit exceed and we need return the last Result's row. - // When user setBatch and the scanner is reopened, the server may return Results that - // user has seen and the last Result can not be seen because the number is not enough. - // So the row keys of results may not be same, we must use the last one. - cache.add(Result.createCursorResult(new Cursor(values[values.length - 1].getRow()))); - break; - } - } - if (countdown <= 0) { - // we have enough result. - closeScannerIfExhausted(regionExhausted); - break; - } - if (remainingResultSize <= 0) { - if (!cache.isEmpty()) { - closeScannerIfExhausted(regionExhausted); - break; - } else { - // we have reached the max result size but we still can not find anything to return to the - // user. Reset the maxResultSize and try again. - remainingResultSize = maxScannerResultSize; - } - } - // we are done with the current region - if (regionExhausted) { - if (!moveToNextRegion()) { - closed = true; - break; - } - } - } - } - - protected void addEstimatedSize(long estimatedHeapSizeOfResult) { - return; - } - - @VisibleForTesting - public int getCacheCount() { - return cache != null ? cache.size() : 0; - } - - @Override - public void close() { - if (callable != null) { - callable.setClose(); - try { - call(callable, caller, scannerTimeout, false); - } catch (UnknownScannerException e) { - // We used to catch this error, interpret, and rethrow. However, we - // have since decided that it's not nice for a scanner's close to - // throw exceptions. Chances are it was just due to lease time out. - LOG.debug("scanner failed to close", e); - } catch (IOException e) { - /* An exception other than UnknownScanner is unexpected. */ - LOG.warn("scanner failed to close.", e); - } - callable = null; - } - closed = true; - } - - @Override - public boolean renewLease() { - if (callable == null) { - return false; - } - // do not return any rows, do not advance the scanner - callable.setRenew(true); - try { - this.caller.callWithoutRetries(callable, this.scannerTimeout); - return true; - } catch (Exception e) { - LOG.debug("scanner failed to renew lease", e); - return false; - } finally { - callable.setRenew(false); - } - } - - protected void initCache() { - initSyncCache(); - } - - @Override - public Result next() throws IOException { - return nextWithSyncCache(); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java deleted file mode 100644 index c7006a87daf4..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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 org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; - -/** - * A RegionServerCallable set to use the Client protocol. - * Also includes some utility methods so can hide protobuf references here rather than have them - * sprinkled about the code base. - * @param - */ -@InterfaceAudience.Private -public abstract class ClientServiceCallable - extends RegionServerCallable { - - public ClientServiceCallable(ConnectionImplementation connection, TableName tableName, byte[] row, - RpcController rpcController, int priority) { - super(connection, tableName, row, rpcController, priority); - } - - @Override - protected void setStubByServiceName(ServerName serviceName) throws IOException { - setStub(getConnection().getClient(serviceName)); - } - - // Below here are simple methods that contain the stub and the rpcController. - protected ClientProtos.GetResponse doGet(ClientProtos.GetRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().get(getRpcController(), request); - } - - protected ClientProtos.MutateResponse doMutate(ClientProtos.MutateRequest request) - throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - return getStub().mutate(getRpcController(), request); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java deleted file mode 100644 index e5d7b97697d0..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter; -import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow; -import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan; - -import java.io.IOException; -import java.util.concurrent.ExecutorService; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; - -/** - * ClientSimpleScanner implements a sync scanner behaviour. - * The cache is a simple list. - * The prefetch is invoked only when the application finished processing the entire cache. - */ -@InterfaceAudience.Private -public class ClientSimpleScanner extends ClientScanner { - public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name, - ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory, - RpcControllerFactory rpcControllerFactory, ExecutorService pool, - int replicaCallTimeoutMicroSecondScan) throws IOException { - super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, - replicaCallTimeoutMicroSecondScan); - } - - @Override - protected boolean setNewStartKey() { - if (noMoreResultsForScan(scan, currentRegion)) { - return false; - } - scan.withStartRow(currentRegion.getEndKey(), true); - return true; - } - - @Override - protected ScannerCallable createScannerCallable() { - if (!scan.includeStartRow() && !isEmptyStartRow(scan.getStartRow())) { - // we have not implemented locate to next row for sync client yet, so here we change the - // inclusive of start row to true. - scan.withStartRow(createClosestRowAfter(scan.getStartRow()), true); - } - return new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - this.rpcControllerFactory); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java index 53859c2ac6a2..9edf8c251ee5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java @@ -44,6 +44,7 @@ public class ConnectionConfiguration { public static final String PRIMARY_SCAN_TIMEOUT_MICROSECOND = "hbase.client.replicaCallTimeout.scan"; public static final int PRIMARY_SCAN_TIMEOUT_MICROSECOND_DEFAULT = 1000000; // 1s + public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity"; private final long writeBufferSize; private final long writeBufferPeriodicFlushTimeoutMs; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index b6d01616a198..2b432e47f6c2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -231,14 +231,6 @@ public static Connection createConnection(Configuration conf, ExecutorService po } } - /** - * Create a {@link ConnectionImplementation}, internal use only. - */ - static ConnectionImplementation createConnectionImpl(Configuration conf, ExecutorService pool, - User user) throws IOException { - return new ConnectionImplementation(conf, pool, user); - } - /** * Call {@link #createAsyncConnection(Configuration)} using default HBaseConfiguration. * @see #createAsyncConnection(Configuration) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java deleted file mode 100644 index fa74b9a8c192..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ /dev/null @@ -1,2178 +0,0 @@ -/** - * - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR; -import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey; -import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts; -import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY; -import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent; -import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsentEx; - -import edu.umd.cs.findbugs.annotations.Nullable; -import java.io.Closeable; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.lang.reflect.UndeclaredThrowableException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Date; -import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.ReentrantLock; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.AuthUtil; -import org.apache.hadoop.hbase.CallQueueTooBigException; -import org.apache.hadoop.hbase.ChoreService; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotEnabledException; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.client.Scan.ReadType; -import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; -import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; -import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; -import org.apache.hadoop.hbase.exceptions.RegionMovedException; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.ipc.RpcClientFactory; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.log.HBaseMarkers; -import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.ReflectionUtils; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Throwables; -import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; - -/** - * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces. - * Encapsulates connection to zookeeper and regionservers. - */ -@edu.umd.cs.findbugs.annotations.SuppressWarnings( - value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", - justification="Access to the conncurrent hash map is under a lock so should be fine.") -@InterfaceAudience.Private -class ConnectionImplementation implements Connection, Closeable { - public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; - private static final Logger LOG = LoggerFactory.getLogger(ConnectionImplementation.class); - - private static final String RESOLVE_HOSTNAME_ON_FAIL_KEY = "hbase.resolve.hostnames.on.failure"; - - private final boolean hostnamesCanChange; - private final long pause; - private final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified - private boolean useMetaReplicas; - private final int metaReplicaCallTimeoutScanInMicroSecond; - private final int numTries; - final int rpcTimeout; - - /** - * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. - * Once it's set under nonceGeneratorCreateLock, it is never unset or changed. - */ - private static volatile NonceGenerator nonceGenerator = null; - /** The nonce generator lock. Only taken when creating Connection, which gets a private copy. */ - private static final Object nonceGeneratorCreateLock = new Object(); - - private final AsyncProcess asyncProcess; - // single tracker per connection - private final ServerStatisticTracker stats; - - private volatile boolean closed; - private volatile boolean aborted; - - // package protected for the tests - ClusterStatusListener clusterStatusListener; - - private final Object metaRegionLock = new Object(); - - private final Object masterLock = new Object(); - - // thread executor shared by all Table instances created - // by this connection - private volatile ThreadPoolExecutor batchPool = null; - // meta thread executor shared by all Table instances created - // by this connection - private volatile ThreadPoolExecutor metaLookupPool = null; - private volatile boolean cleanupPool = false; - - private final Configuration conf; - - // cache the configuration value for tables so that we can avoid calling - // the expensive Configuration to fetch the value multiple times. - private final ConnectionConfiguration connectionConfig; - - // Client rpc instance. - private final RpcClient rpcClient; - - private final MetaCache metaCache; - private final MetricsConnection metrics; - - protected User user; - - private final RpcRetryingCallerFactory rpcCallerFactory; - - private final RpcControllerFactory rpcControllerFactory; - - private final RetryingCallerInterceptor interceptor; - - /** - * Cluster registry of basic info such as clusterid and meta region location. - */ - private final AsyncRegistry registry; - - private final ClientBackoffPolicy backoffPolicy; - - /** - * Allow setting an alternate BufferedMutator implementation via - * config. If null, use default. - */ - private final String alternateBufferedMutatorClassName; - - /** lock guards against multiple threads trying to query the meta region at the same time */ - private final ReentrantLock userRegionLock = new ReentrantLock(); - - private ChoreService authService; - - /** - * constructor - * @param conf Configuration object - */ - ConnectionImplementation(Configuration conf, ExecutorService pool, User user) throws IOException { - this.conf = conf; - this.user = user; - if (user != null && user.isLoginFromKeytab()) { - spawnRenewalChore(user.getUGI()); - } - this.batchPool = (ThreadPoolExecutor) pool; - this.connectionConfig = new ConnectionConfiguration(conf); - this.closed = false; - this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); - if (configuredPauseForCQTBE < pause) { - LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " - + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE - + ", will use " + pause + " instead."); - this.pauseForCQTBE = pause; - } else { - this.pauseForCQTBE = configuredPauseForCQTBE; - } - this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS, - HConstants.DEFAULT_USE_META_REPLICAS); - this.metaReplicaCallTimeoutScanInMicroSecond = - connectionConfig.getMetaReplicaCallTimeoutMicroSecondScan(); - - // how many times to try, one more than max *retry* time - this.numTries = retries2Attempts(connectionConfig.getRetriesNumber()); - this.rpcTimeout = conf.getInt( - HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - if (conf.getBoolean(NonceGenerator.CLIENT_NONCES_ENABLED_KEY, true)) { - synchronized (nonceGeneratorCreateLock) { - if (nonceGenerator == null) { - nonceGenerator = PerClientRandomNonceGenerator.get(); - } - } - } else { - nonceGenerator = NO_NONCE_GENERATOR; - } - - this.stats = ServerStatisticTracker.create(conf); - this.interceptor = (new RetryingCallerInterceptorFactory(conf)).build(); - this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); - this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, interceptor, this.stats); - this.backoffPolicy = ClientBackoffPolicyFactory.create(conf); - this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory); - if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) { - this.metrics = - new MetricsConnection(this.toString(), this::getBatchPool, this::getMetaLookupPool); - } else { - this.metrics = null; - } - this.metaCache = new MetaCache(this.metrics); - - boolean shouldListen = conf.getBoolean(HConstants.STATUS_PUBLISHED, - HConstants.STATUS_PUBLISHED_DEFAULT); - this.hostnamesCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true); - Class listenerClass = - conf.getClass(ClusterStatusListener.STATUS_LISTENER_CLASS, - ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS, - ClusterStatusListener.Listener.class); - - // Is there an alternate BufferedMutator to use? - this.alternateBufferedMutatorClassName = - this.conf.get(BufferedMutator.CLASSNAME_KEY); - - try { - this.registry = AsyncRegistryFactory.getRegistry(conf); - retrieveClusterId(); - - this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics); - - // Do we publish the status? - if (shouldListen) { - if (listenerClass == null) { - LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " + - ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status"); - } else { - clusterStatusListener = new ClusterStatusListener( - new ClusterStatusListener.DeadServerHandler() { - @Override - public void newDead(ServerName sn) { - clearCaches(sn); - rpcClient.cancelConnections(sn); - } - }, conf, listenerClass); - } - } - } catch (Throwable e) { - // avoid leaks: registry, rpcClient, ... - LOG.debug("connection construction failed", e); - close(); - throw e; - } - } - - private void spawnRenewalChore(final UserGroupInformation user) { - authService = new ChoreService("Relogin service"); - authService.scheduleChore(AuthUtil.getAuthRenewalChore(user)); - } - - /** - * @param useMetaReplicas - */ - @VisibleForTesting - void setUseMetaReplicas(final boolean useMetaReplicas) { - this.useMetaReplicas = useMetaReplicas; - } - - /** - * @param conn The connection for which to replace the generator. - * @param cnm Replaces the nonce generator used, for testing. - * @return old nonce generator. - */ - @VisibleForTesting - static NonceGenerator injectNonceGeneratorForTesting( - ConnectionImplementation conn, NonceGenerator cnm) { - NonceGenerator ng = conn.getNonceGenerator(); - LOG.warn("Nonce generator is being replaced by test code for " - + cnm.getClass().getName()); - nonceGenerator = cnm; - return ng; - } - - @Override - public Table getTable(TableName tableName) throws IOException { - return getTable(tableName, getBatchPool()); - } - - @Override - public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) { - return new TableBuilderBase(tableName, connectionConfig) { - - @Override - public Table build() { - return new HTable(ConnectionImplementation.this, this, rpcCallerFactory, - rpcControllerFactory, pool); - } - }; - } - - @Override - public BufferedMutator getBufferedMutator(BufferedMutatorParams params) { - if (params.getTableName() == null) { - throw new IllegalArgumentException("TableName cannot be null."); - } - if (params.getPool() == null) { - params.pool(HTable.getDefaultExecutor(getConfiguration())); - } - if (params.getWriteBufferSize() == BufferedMutatorParams.UNSET) { - params.writeBufferSize(connectionConfig.getWriteBufferSize()); - } - if (params.getWriteBufferPeriodicFlushTimeoutMs() == BufferedMutatorParams.UNSET) { - params.setWriteBufferPeriodicFlushTimeoutMs( - connectionConfig.getWriteBufferPeriodicFlushTimeoutMs()); - } - if (params.getWriteBufferPeriodicFlushTimerTickMs() == BufferedMutatorParams.UNSET) { - params.setWriteBufferPeriodicFlushTimerTickMs( - connectionConfig.getWriteBufferPeriodicFlushTimerTickMs()); - } - if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) { - params.maxKeyValueSize(connectionConfig.getMaxKeyValueSize()); - } - // Look to see if an alternate BufferedMutation implementation is wanted. - // Look in params and in config. If null, use default. - String implementationClassName = params.getImplementationClassName(); - if (implementationClassName == null) { - implementationClassName = this.alternateBufferedMutatorClassName; - } - if (implementationClassName == null) { - return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params); - } - try { - return (BufferedMutator)ReflectionUtils.newInstance(Class.forName(implementationClassName), - this, rpcCallerFactory, rpcControllerFactory, params); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - - @Override - public RegionLocator getRegionLocator(TableName tableName) throws IOException { - return new HRegionLocator(tableName, this); - } - - @Override - public Admin getAdmin() throws IOException { - return new HBaseAdmin(this); - } - - @Override - public Hbck getHbck() throws IOException { - return getHbck(get(registry.getMasterAddress())); - } - - @Override - public Hbck getHbck(ServerName masterServer) throws IOException { - checkClosed(); - if (isDeadServer(masterServer)) { - throw new RegionServerStoppedException(masterServer + " is dead."); - } - String key = getStubKey(MasterProtos.HbckService.BlockingInterface.class.getName(), - masterServer, this.hostnamesCanChange); - - return new HBaseHbck( - (MasterProtos.HbckService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { - BlockingRpcChannel channel = - this.rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout); - return MasterProtos.HbckService.newBlockingStub(channel); - }), rpcControllerFactory); - } - - /** - * @return the MetricsConnection instance associated with this connection. - */ - public MetricsConnection getConnectionMetrics() { - return this.metrics; - } - - private ThreadPoolExecutor getBatchPool() { - if (batchPool == null) { - synchronized (this) { - if (batchPool == null) { - int threads = conf.getInt("hbase.hconnection.threads.max", 256); - this.batchPool = getThreadPool(threads, threads, "-shared", null); - this.cleanupPool = true; - } - } - } - return this.batchPool; - } - - private ThreadPoolExecutor getThreadPool(int maxThreads, int coreThreads, String nameHint, - BlockingQueue passedWorkQueue) { - // shared HTable thread executor not yet initialized - return ConnectionUtils.getThreadPool(conf, maxThreads, coreThreads, () -> toString() + nameHint, - passedWorkQueue); - } - - private ThreadPoolExecutor getMetaLookupPool() { - if (this.metaLookupPool == null) { - synchronized (this) { - if (this.metaLookupPool == null) { - //Some of the threads would be used for meta replicas - //To start with, threads.max.core threads can hit the meta (including replicas). - //After that, requests will get queued up in the passed queue, and only after - //the queue is full, a new thread will be started - int threads = conf.getInt("hbase.hconnection.meta.lookup.threads.max", 128); - this.metaLookupPool = getThreadPool( - threads, - threads, - "-metaLookup-shared-", new LinkedBlockingQueue<>()); - } - } - } - return this.metaLookupPool; - } - - protected ExecutorService getCurrentMetaLookupPool() { - return metaLookupPool; - } - - protected ExecutorService getCurrentBatchPool() { - return batchPool; - } - - private void shutdownPools() { - if (this.cleanupPool && this.batchPool != null && !this.batchPool.isShutdown()) { - ConnectionUtils.shutdownPool(this.batchPool); - } - if (this.metaLookupPool != null && !this.metaLookupPool.isShutdown()) { - ConnectionUtils.shutdownPool(this.metaLookupPool); - } - } - - /** - * For tests only. - */ - @VisibleForTesting - RpcClient getRpcClient() { - return rpcClient; - } - - /** - * An identifier that will remain the same for a given connection. - */ - @Override - public String toString(){ - return "hconnection-0x" + Integer.toHexString(hashCode()); - } - - protected String clusterId = null; - - protected void retrieveClusterId() { - if (clusterId != null) { - return; - } - try { - this.clusterId = this.registry.getClusterId().get(); - } catch (InterruptedException | ExecutionException e) { - LOG.warn("Retrieve cluster id failed", e); - } - if (clusterId == null) { - clusterId = HConstants.CLUSTER_ID_DEFAULT; - LOG.debug("clusterid came back null, using default " + clusterId); - } - } - - @Override - public Configuration getConfiguration() { - return this.conf; - } - - private void checkClosed() throws DoNotRetryIOException { - if (this.closed) { - throw new DoNotRetryIOException(toString() + " closed"); - } - } - - /** - * @return true if the master is running, throws an exception otherwise - * @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running - * @deprecated this has been deprecated without a replacement - */ - @Deprecated - public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException { - // When getting the master connection, we check it's running, - // so if there is no exception, it means we've been able to get a - // connection on a running master - MasterKeepAliveConnection m; - try { - m = getKeepAliveMasterService(); - } catch (IOException e) { - throw new MasterNotRunningException(e); - } - m.close(); - return true; - } - - /** - * Find region location hosting passed row - * @param tableName table name - * @param row Row to find. - * @param reload If true do not use cache, otherwise bypass. - * @return Location of row. - * @throws IOException if a remote or network exception occurs - */ - HRegionLocation getRegionLocation(final TableName tableName, final byte[] row, boolean reload) - throws IOException { - return reload ? relocateRegion(tableName, row) : locateRegion(tableName, row); - } - - /** - * A table that isTableEnabled == false and isTableDisabled == false - * is possible. This happens when a table has a lot of regions - * that must be processed. - * @param tableName table name - * @return true if the table is enabled, false otherwise - * @throws IOException if a remote or network exception occurs - */ - public boolean isTableEnabled(TableName tableName) throws IOException { - return getTableState(tableName).inStates(TableState.State.ENABLED); - } - - /** - * @param tableName table name - * @return true if the table is disabled, false otherwise - * @throws IOException if a remote or network exception occurs - */ - public boolean isTableDisabled(TableName tableName) throws IOException { - return getTableState(tableName).inStates(TableState.State.DISABLED); - } - - /** - * Use this api to check if the table has been created with the specified number of - * splitkeys which was used while creating the given table. - * Note : If this api is used after a table's region gets splitted, the api may return - * false. - * @param tableName - * tableName - * @param splitKeys - * splitKeys used while creating table - * @throws IOException - * if a remote or network exception occurs - */ - public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys) - throws IOException { - checkClosed(); - try { - if (!isTableEnabled(tableName)) { - LOG.debug("Table {} not enabled", tableName); - return false; - } - List> locations = - MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true); - - int notDeployed = 0; - int regionCount = 0; - for (Pair pair : locations) { - RegionInfo info = pair.getFirst(); - if (pair.getSecond() == null) { - LOG.debug("Table {} has not deployed region {}", tableName, - pair.getFirst().getEncodedName()); - notDeployed++; - } else if (splitKeys != null - && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { - for (byte[] splitKey : splitKeys) { - // Just check if the splitkey is available - if (Bytes.equals(info.getStartKey(), splitKey)) { - regionCount++; - break; - } - } - } else { - // Always empty start row should be counted - regionCount++; - } - } - if (notDeployed > 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("Table {} has {} regions not deployed", tableName, notDeployed); - } - return false; - } else if (splitKeys != null && regionCount != splitKeys.length + 1) { - if (LOG.isDebugEnabled()) { - LOG.debug("Table {} expected to have {} regions, but only {} available", tableName, - splitKeys.length + 1, regionCount); - } - return false; - } else { - LOG.trace("Table {} should be available", tableName); - return true; - } - } catch (TableNotFoundException tnfe) { - LOG.warn("Table {} does not exist", tableName); - return false; - } - } - - private boolean isDeadServer(ServerName sn) { - if (clusterStatusListener == null) { - return false; - } else { - return clusterStatusListener.isDeadServer(sn); - } - } - - /** - * Gets the locations of all regions in the specified table, tableName. - * @param tableName table to get regions of - * @return list of region locations for all regions of table - * @throws IOException if IO failure occurs - */ - List locateRegions(TableName tableName) throws IOException { - return locateRegions(tableName, false, true); - } - - /** - * Gets the locations of all regions in the specified table, tableName. - * @param tableName table to get regions of - * @param useCache Should we use the cache to retrieve the region information. - * @param offlined True if we are to include offlined regions, false and we'll leave out offlined - * regions from returned list. - * @return list of region locations for all regions of table - * @throws IOException if IO failure occurs - */ - List locateRegions(TableName tableName, boolean useCache, - boolean offlined) throws IOException { - List regions; - if (TableName.isMetaTableName(tableName)) { - regions = Collections.singletonList(RegionInfoBuilder.FIRST_META_REGIONINFO); - } else { - regions = MetaTableAccessor.getTableRegions(this, tableName, !offlined); - } - List locations = new ArrayList<>(); - for (RegionInfo regionInfo : regions) { - if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) { - continue; - } - RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true); - if (list != null) { - for (HRegionLocation loc : list.getRegionLocations()) { - if (loc != null) { - locations.add(loc); - } - } - } - } - return locations; - } - - /** - * Find the location of the region of tableName that row lives in. - * @param tableName name of the table row is in - * @param row row key you're trying to find the region of - * @return HRegionLocation that describes where to find the region in question - * @throws IOException if a remote or network exception occurs - */ - HRegionLocation locateRegion(final TableName tableName, final byte[] row) throws IOException { - RegionLocations locations = locateRegion(tableName, row, true, true); - return locations == null ? null : locations.getRegionLocation(); - } - - /** - * Find the location of the region of tableName that row lives in, ignoring any - * value that might be in the cache. - * @param tableName name of the table row is in - * @param row row key you're trying to find the region of - * @return HRegionLocation that describes where to find the region in question - * @throws IOException if a remote or network exception occurs - */ - HRegionLocation relocateRegion(final TableName tableName, final byte[] row) throws IOException { - RegionLocations locations = - relocateRegion(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID); - return locations == null ? null - : locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID); - } - - /** - * Find the location of the region of tableName that row - * lives in, ignoring any value that might be in the cache. - * @param tableName name of the table row is in - * @param row row key you're trying to find the region of - * @param replicaId the replicaId of the region - * @return RegionLocations that describe where to find the region in - * question - * @throws IOException if a remote or network exception occurs - */ - RegionLocations relocateRegion(final TableName tableName, - final byte [] row, int replicaId) throws IOException{ - // Since this is an explicit request not to use any caching, finding - // disabled tables should not be desirable. This will ensure that an exception is thrown when - // the first time a disabled table is interacted with. - if (!tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) { - throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled."); - } - - return locateRegion(tableName, row, false, true, replicaId); - } - - /** - * @param tableName table to get regions of - * @param row the row - * @param useCache Should we use the cache to retrieve the region information. - * @param retry do we retry - * @return region locations for this row. - * @throws IOException if IO failure occurs - */ - RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache, - boolean retry) throws IOException { - return locateRegion(tableName, row, useCache, retry, RegionReplicaUtil.DEFAULT_REPLICA_ID); - } - - /** - * @param tableName table to get regions of - * @param row the row - * @param useCache Should we use the cache to retrieve the region information. - * @param retry do we retry - * @param replicaId the replicaId for the region - * @return region locations for this row. - * @throws IOException if IO failure occurs - */ - RegionLocations locateRegion(final TableName tableName, final byte[] row, boolean useCache, - boolean retry, int replicaId) throws IOException { - checkClosed(); - if (tableName == null || tableName.getName().length == 0) { - throw new IllegalArgumentException("table name cannot be null or zero length"); - } - if (tableName.equals(TableName.META_TABLE_NAME)) { - return locateMeta(tableName, useCache, replicaId); - } else { - // Region not in the cache - have to go to the meta RS - return locateRegionInMeta(tableName, row, useCache, retry, replicaId); - } - } - - private RegionLocations locateMeta(final TableName tableName, - boolean useCache, int replicaId) throws IOException { - // HBASE-10785: We cache the location of the META itself, so that we are not overloading - // zookeeper with one request for every region lookup. We cache the META with empty row - // key in MetaCache. - byte[] metaCacheKey = HConstants.EMPTY_START_ROW; // use byte[0] as the row for meta - RegionLocations locations = null; - if (useCache) { - locations = getCachedLocation(tableName, metaCacheKey); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - - // only one thread should do the lookup. - synchronized (metaRegionLock) { - // Check the cache again for a hit in case some other thread made the - // same query while we were waiting on the lock. - if (useCache) { - locations = getCachedLocation(tableName, metaCacheKey); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - - // Look up from zookeeper - locations = get(this.registry.getMetaRegionLocation()); - if (locations != null) { - cacheLocation(tableName, locations); - } - } - return locations; - } - - /** - * Search the hbase:meta table for the HRegionLocation info that contains the table and row we're - * seeking. - */ - private RegionLocations locateRegionInMeta(TableName tableName, byte[] row, boolean useCache, - boolean retry, int replicaId) throws IOException { - // If we are supposed to be using the cache, look in the cache to see if we already have the - // region. - if (useCache) { - RegionLocations locations = getCachedLocation(tableName, row); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - // build the key of the meta region we should be looking for. - // the extra 9's on the end are necessary to allow "exact" matches - // without knowing the precise region names. - byte[] metaStartKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false); - byte[] metaStopKey = - RegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, "", false); - Scan s = new Scan().withStartRow(metaStartKey).withStopRow(metaStopKey, true) - .addFamily(HConstants.CATALOG_FAMILY).setReversed(true).setCaching(5) - .setReadType(ReadType.PREAD); - if (this.useMetaReplicas) { - s.setConsistency(Consistency.TIMELINE); - } - int maxAttempts = (retry ? numTries : 1); - boolean relocateMeta = false; - for (int tries = 0; ; tries++) { - if (tries >= maxAttempts) { - throw new NoServerForRegionException("Unable to find region for " - + Bytes.toStringBinary(row) + " in " + tableName + " after " + tries + " tries."); - } - if (useCache) { - RegionLocations locations = getCachedLocation(tableName, row); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } else { - // If we are not supposed to be using the cache, delete any existing cached location - // so it won't interfere. - // We are only supposed to clean the cache for the specific replicaId - metaCache.clearCache(tableName, row, replicaId); - } - // Query the meta region - long pauseBase = this.pause; - userRegionLock.lock(); - try { - if (useCache) {// re-check cache after get lock - RegionLocations locations = getCachedLocation(tableName, row); - if (locations != null && locations.getRegionLocation(replicaId) != null) { - return locations; - } - } - if (relocateMeta) { - relocateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW, - RegionInfo.DEFAULT_REPLICA_ID); - } - s.resetMvccReadPoint(); - try (ReversedClientScanner rcs = - new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory, - rpcControllerFactory, getMetaLookupPool(), metaReplicaCallTimeoutScanInMicroSecond)) { - boolean tableNotFound = true; - for (;;) { - Result regionInfoRow = rcs.next(); - if (regionInfoRow == null) { - if (tableNotFound) { - throw new TableNotFoundException(tableName); - } else { - throw new IOException( - "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName); - } - } - tableNotFound = false; - // convert the row result into the HRegionLocation we need! - RegionLocations locations = MetaTableAccessor.getRegionLocations(regionInfoRow); - if (locations == null || locations.getRegionLocation(replicaId) == null) { - throw new IOException("RegionInfo null in " + tableName + ", row=" + regionInfoRow); - } - RegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegion(); - if (regionInfo == null) { - throw new IOException("RegionInfo null or empty in " + TableName.META_TABLE_NAME + - ", row=" + regionInfoRow); - } - // See HBASE-20182. It is possible that we locate to a split parent even after the - // children are online, so here we need to skip this region and go to the next one. - if (regionInfo.isSplitParent()) { - continue; - } - if (regionInfo.isOffline()) { - throw new RegionOfflineException("Region offline; disable table call? " + - regionInfo.getRegionNameAsString()); - } - // It is possible that the split children have not been online yet and we have skipped - // the parent in the above condition, so we may have already reached a region which does - // not contains us. - if (!regionInfo.containsRow(row)) { - throw new IOException( - "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName); - } - ServerName serverName = locations.getRegionLocation(replicaId).getServerName(); - if (serverName == null) { - throw new NoServerForRegionException("No server address listed in " + - TableName.META_TABLE_NAME + " for region " + regionInfo.getRegionNameAsString() + - " containing row " + Bytes.toStringBinary(row)); - } - if (isDeadServer(serverName)) { - throw new RegionServerStoppedException( - "hbase:meta says the region " + regionInfo.getRegionNameAsString() + - " is managed by the server " + serverName + ", but it is dead."); - } - // Instantiate the location - cacheLocation(tableName, locations); - return locations; - } - } - } catch (TableNotFoundException e) { - // if we got this error, probably means the table just plain doesn't - // exist. rethrow the error immediately. this should always be coming - // from the HTable constructor. - throw e; - } catch (IOException e) { - ExceptionUtil.rethrowIfInterrupt(e); - if (e instanceof RemoteException) { - e = ((RemoteException)e).unwrapRemoteException(); - } - if (e instanceof CallQueueTooBigException) { - // Give a special check on CallQueueTooBigException, see #HBASE-17114 - pauseBase = this.pauseForCQTBE; - } - if (tries < maxAttempts - 1) { - LOG.debug("locateRegionInMeta parentTable='{}', attempt={} of {} failed; retrying " + - "after sleep of {}", TableName.META_TABLE_NAME, tries, maxAttempts, maxAttempts, e); - } else { - throw e; - } - // Only relocate the parent region if necessary - relocateMeta = - !(e instanceof RegionOfflineException || e instanceof NoServerForRegionException); - } finally { - userRegionLock.unlock(); - } - try{ - Thread.sleep(ConnectionUtils.getPauseTime(pauseBase, tries)); - } catch (InterruptedException e) { - throw new InterruptedIOException("Giving up trying to location region in " + - "meta: thread is interrupted."); - } - } - } - - /** - * Put a newly discovered HRegionLocation into the cache. - * @param tableName The table name. - * @param location the new location - */ - void cacheLocation(final TableName tableName, final RegionLocations location) { - metaCache.cacheLocation(tableName, location); - } - - /** - * Search the cache for a location that fits our table and row key. - * Return null if no suitable region is located. - * @return Null or region location found in cache. - */ - RegionLocations getCachedLocation(final TableName tableName, - final byte [] row) { - return metaCache.getCachedLocation(tableName, row); - } - - void clearRegionCache(final TableName tableName, byte[] row) { - metaCache.clearCache(tableName, row); - } - - /** - * Clear any caches that pertain to server name sn. - * @param sn A server name - */ - void clearCaches(final ServerName serverName) { - metaCache.clearCache(serverName); - } - - - /** - * Allows flushing the region cache. - */ - @Override - public void clearRegionLocationCache() { - metaCache.clearCache(); - } - - /** - * Allows flushing the region cache of all locations that pertain to tableName - * @param tableName Name of the table whose regions we are to remove from cache. - */ - void clearRegionCache(final TableName tableName) { - metaCache.clearCache(tableName); - } - - /** - * Put a newly discovered HRegionLocation into the cache. - * @param tableName The table name. - * @param source the source of the new location, if it's not coming from meta - * @param location the new location - */ - private void cacheLocation(final TableName tableName, final ServerName source, - final HRegionLocation location) { - metaCache.cacheLocation(tableName, source, location); - } - - // Map keyed by service name + regionserver to service stub implementation - private final ConcurrentMap stubs = new ConcurrentHashMap<>(); - - /** - * State of the MasterService connection/setup. - */ - static class MasterServiceState { - Connection connection; - - MasterProtos.MasterService.BlockingInterface stub; - int userCount; - - MasterServiceState(final Connection connection) { - super(); - this.connection = connection; - } - - @Override - public String toString() { - return "MasterService"; - } - - Object getStub() { - return this.stub; - } - - void clearStub() { - this.stub = null; - } - - boolean isMasterRunning() throws IOException { - MasterProtos.IsMasterRunningResponse response = null; - try { - response = this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - return response != null? response.getIsMasterRunning(): false; - } - } - - /** - * The record of errors for servers. - */ - static class ServerErrorTracker { - // We need a concurrent map here, as we could have multiple threads updating it in parallel. - private final ConcurrentMap errorsByServer = new ConcurrentHashMap<>(); - private final long canRetryUntil; - private final int maxTries;// max number to try - private final long startTrackingTime; - - /** - * Constructor - * @param timeout how long to wait before timeout, in unit of millisecond - * @param maxTries how many times to try - */ - public ServerErrorTracker(long timeout, int maxTries) { - this.maxTries = maxTries; - this.canRetryUntil = EnvironmentEdgeManager.currentTime() + timeout; - this.startTrackingTime = new Date().getTime(); - } - - /** - * We stop to retry when we have exhausted BOTH the number of tries and the time allocated. - * @param numAttempt how many times we have tried by now - */ - boolean canTryMore(int numAttempt) { - // If there is a single try we must not take into account the time. - return numAttempt < maxTries || (maxTries > 1 && - EnvironmentEdgeManager.currentTime() < this.canRetryUntil); - } - - /** - * Calculates the back-off time for a retrying request to a particular server. - * - * @param server The server in question. - * @param basePause The default hci pause. - * @return The time to wait before sending next request. - */ - long calculateBackoffTime(ServerName server, long basePause) { - long result; - ServerErrors errorStats = errorsByServer.get(server); - if (errorStats != null) { - result = ConnectionUtils.getPauseTime(basePause, Math.max(0, errorStats.getCount() - 1)); - } else { - result = 0; // yes, if the server is not in our list we don't wait before retrying. - } - return result; - } - - /** - * Reports that there was an error on the server to do whatever bean-counting necessary. - * @param server The server in question. - */ - void reportServerError(ServerName server) { - computeIfAbsent(errorsByServer, server, ServerErrors::new).addError(); - } - - long getStartTrackingTime() { - return startTrackingTime; - } - - /** - * The record of errors for a server. - */ - private static class ServerErrors { - private final AtomicInteger retries = new AtomicInteger(0); - - public int getCount() { - return retries.get(); - } - - public void addError() { - retries.incrementAndGet(); - } - } - } - - /** - * Class to make a MasterServiceStubMaker stub. - */ - private final class MasterServiceStubMaker { - - private void isMasterRunning(MasterProtos.MasterService.BlockingInterface stub) - throws IOException { - try { - stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); - } catch (ServiceException e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - /** - * Create a stub. Try once only. It is not typed because there is no common type to protobuf - * services nor their interfaces. Let the caller do appropriate casting. - * @return A stub for master services. - */ - private MasterProtos.MasterService.BlockingInterface makeStubNoRetries() - throws IOException, KeeperException { - ServerName sn = get(registry.getMasterAddress()); - if (sn == null) { - String msg = "ZooKeeper available but no active master location found"; - LOG.info(msg); - throw new MasterNotRunningException(msg); - } - if (isDeadServer(sn)) { - throw new MasterNotRunningException(sn + " is dead."); - } - // Use the security info interface name as our stub key - String key = - getStubKey(MasterProtos.MasterService.getDescriptor().getName(), sn, hostnamesCanChange); - MasterProtos.MasterService.BlockingInterface stub = - (MasterProtos.MasterService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { - BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); - return MasterProtos.MasterService.newBlockingStub(channel); - }); - isMasterRunning(stub); - return stub; - } - - /** - * Create a stub against the master. Retry if necessary. - * @return A stub to do intf against the master - * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running - */ - MasterProtos.MasterService.BlockingInterface makeStub() throws IOException { - // The lock must be at the beginning to prevent multiple master creations - // (and leaks) in a multithread context - synchronized (masterLock) { - Exception exceptionCaught = null; - if (!closed) { - try { - return makeStubNoRetries(); - } catch (IOException e) { - exceptionCaught = e; - } catch (KeeperException e) { - exceptionCaught = e; - } - throw new MasterNotRunningException(exceptionCaught); - } else { - throw new DoNotRetryIOException("Connection was closed while trying to get master"); - } - } - } - } - - /** - * Get the admin service for master. - */ - public AdminProtos.AdminService.BlockingInterface getAdminForMaster() throws IOException { - return getAdmin(get(registry.getMasterAddress())); - } - - /** - * Establishes a connection to the region server at the specified address. - * @param serverName the region server to connect to - * @return proxy for HRegionServer - * @throws IOException if a remote or network exception occurs - */ - public AdminProtos.AdminService.BlockingInterface getAdmin(ServerName serverName) - throws IOException { - checkClosed(); - if (isDeadServer(serverName)) { - throw new RegionServerStoppedException(serverName + " is dead."); - } - String key = getStubKey(AdminProtos.AdminService.BlockingInterface.class.getName(), serverName, - this.hostnamesCanChange); - return (AdminProtos.AdminService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { - BlockingRpcChannel channel = - this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout); - return AdminProtos.AdminService.newBlockingStub(channel); - }); - } - - /** - * Establishes a connection to the region server at the specified address, and returns a region - * client protocol. - * @param serverName the region server to connect to - * @return ClientProtocol proxy for RegionServer - * @throws IOException if a remote or network exception occurs - */ - public BlockingInterface getClient(ServerName serverName) throws IOException { - checkClosed(); - if (isDeadServer(serverName)) { - throw new RegionServerStoppedException(serverName + " is dead."); - } - String key = getStubKey(ClientProtos.ClientService.BlockingInterface.class.getName(), - serverName, this.hostnamesCanChange); - return (ClientProtos.ClientService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { - BlockingRpcChannel channel = - this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout); - return ClientProtos.ClientService.newBlockingStub(channel); - }); - } - - final MasterServiceState masterServiceState = new MasterServiceState(this); - - public MasterKeepAliveConnection getMaster() throws IOException { - return getKeepAliveMasterService(); - } - - private void resetMasterServiceState(final MasterServiceState mss) { - mss.userCount++; - } - - private MasterKeepAliveConnection getKeepAliveMasterService() throws IOException { - synchronized (masterLock) { - if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) { - MasterServiceStubMaker stubMaker = new MasterServiceStubMaker(); - this.masterServiceState.stub = stubMaker.makeStub(); - } - resetMasterServiceState(this.masterServiceState); - } - // Ugly delegation just so we can add in a Close method. - final MasterProtos.MasterService.BlockingInterface stub = this.masterServiceState.stub; - return new MasterKeepAliveConnection() { - MasterServiceState mss = masterServiceState; - - @Override - public MasterProtos.AbortProcedureResponse abortProcedure( - RpcController controller, - MasterProtos.AbortProcedureRequest request) throws ServiceException { - return stub.abortProcedure(controller, request); - } - - @Override - public MasterProtos.GetProceduresResponse getProcedures( - RpcController controller, - MasterProtos.GetProceduresRequest request) throws ServiceException { - return stub.getProcedures(controller, request); - } - - @Override - public MasterProtos.GetLocksResponse getLocks( - RpcController controller, - MasterProtos.GetLocksRequest request) throws ServiceException { - return stub.getLocks(controller, request); - } - - @Override - public MasterProtos.AddColumnResponse addColumn( - RpcController controller, - MasterProtos.AddColumnRequest request) throws ServiceException { - return stub.addColumn(controller, request); - } - - @Override - public MasterProtos.DeleteColumnResponse deleteColumn(RpcController controller, - MasterProtos.DeleteColumnRequest request) - throws ServiceException { - return stub.deleteColumn(controller, request); - } - - @Override - public MasterProtos.ModifyColumnResponse modifyColumn(RpcController controller, - MasterProtos.ModifyColumnRequest request) - throws ServiceException { - return stub.modifyColumn(controller, request); - } - - @Override - public MasterProtos.MoveRegionResponse moveRegion(RpcController controller, - MasterProtos.MoveRegionRequest request) throws ServiceException { - return stub.moveRegion(controller, request); - } - - @Override - public MasterProtos.MergeTableRegionsResponse mergeTableRegions( - RpcController controller, MasterProtos.MergeTableRegionsRequest request) - throws ServiceException { - return stub.mergeTableRegions(controller, request); - } - - @Override - public MasterProtos.AssignRegionResponse assignRegion(RpcController controller, - MasterProtos.AssignRegionRequest request) throws ServiceException { - return stub.assignRegion(controller, request); - } - - @Override - public MasterProtos.UnassignRegionResponse unassignRegion(RpcController controller, - MasterProtos.UnassignRegionRequest request) throws ServiceException { - return stub.unassignRegion(controller, request); - } - - @Override - public MasterProtos.OfflineRegionResponse offlineRegion(RpcController controller, - MasterProtos.OfflineRegionRequest request) throws ServiceException { - return stub.offlineRegion(controller, request); - } - - @Override - public MasterProtos.SplitTableRegionResponse splitRegion(RpcController controller, - MasterProtos.SplitTableRegionRequest request) throws ServiceException { - return stub.splitRegion(controller, request); - } - - @Override - public MasterProtos.DeleteTableResponse deleteTable(RpcController controller, - MasterProtos.DeleteTableRequest request) throws ServiceException { - return stub.deleteTable(controller, request); - } - - @Override - public MasterProtos.TruncateTableResponse truncateTable(RpcController controller, - MasterProtos.TruncateTableRequest request) throws ServiceException { - return stub.truncateTable(controller, request); - } - - @Override - public MasterProtos.EnableTableResponse enableTable(RpcController controller, - MasterProtos.EnableTableRequest request) throws ServiceException { - return stub.enableTable(controller, request); - } - - @Override - public MasterProtos.DisableTableResponse disableTable(RpcController controller, - MasterProtos.DisableTableRequest request) throws ServiceException { - return stub.disableTable(controller, request); - } - - @Override - public MasterProtos.ModifyTableResponse modifyTable(RpcController controller, - MasterProtos.ModifyTableRequest request) throws ServiceException { - return stub.modifyTable(controller, request); - } - - @Override - public MasterProtos.CreateTableResponse createTable(RpcController controller, - MasterProtos.CreateTableRequest request) throws ServiceException { - return stub.createTable(controller, request); - } - - @Override - public MasterProtos.ShutdownResponse shutdown(RpcController controller, - MasterProtos.ShutdownRequest request) throws ServiceException { - return stub.shutdown(controller, request); - } - - @Override - public MasterProtos.StopMasterResponse stopMaster(RpcController controller, - MasterProtos.StopMasterRequest request) throws ServiceException { - return stub.stopMaster(controller, request); - } - - @Override - public MasterProtos.IsInMaintenanceModeResponse isMasterInMaintenanceMode( - final RpcController controller, - final MasterProtos.IsInMaintenanceModeRequest request) throws ServiceException { - return stub.isMasterInMaintenanceMode(controller, request); - } - - @Override - public MasterProtos.BalanceResponse balance(RpcController controller, - MasterProtos.BalanceRequest request) throws ServiceException { - return stub.balance(controller, request); - } - - @Override - public MasterProtos.SetBalancerRunningResponse setBalancerRunning( - RpcController controller, MasterProtos.SetBalancerRunningRequest request) - throws ServiceException { - return stub.setBalancerRunning(controller, request); - } - - @Override - public NormalizeResponse normalize(RpcController controller, - NormalizeRequest request) throws ServiceException { - return stub.normalize(controller, request); - } - - @Override - public SetNormalizerRunningResponse setNormalizerRunning( - RpcController controller, SetNormalizerRunningRequest request) - throws ServiceException { - return stub.setNormalizerRunning(controller, request); - } - - @Override - public MasterProtos.RunCatalogScanResponse runCatalogScan(RpcController controller, - MasterProtos.RunCatalogScanRequest request) throws ServiceException { - return stub.runCatalogScan(controller, request); - } - - @Override - public MasterProtos.EnableCatalogJanitorResponse enableCatalogJanitor( - RpcController controller, MasterProtos.EnableCatalogJanitorRequest request) - throws ServiceException { - return stub.enableCatalogJanitor(controller, request); - } - - @Override - public MasterProtos.IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled( - RpcController controller, MasterProtos.IsCatalogJanitorEnabledRequest request) - throws ServiceException { - return stub.isCatalogJanitorEnabled(controller, request); - } - - @Override - public MasterProtos.RunCleanerChoreResponse runCleanerChore(RpcController controller, - MasterProtos.RunCleanerChoreRequest request) - throws ServiceException { - return stub.runCleanerChore(controller, request); - } - - @Override - public MasterProtos.SetCleanerChoreRunningResponse setCleanerChoreRunning( - RpcController controller, MasterProtos.SetCleanerChoreRunningRequest request) - throws ServiceException { - return stub.setCleanerChoreRunning(controller, request); - } - - @Override - public MasterProtos.IsCleanerChoreEnabledResponse isCleanerChoreEnabled( - RpcController controller, MasterProtos.IsCleanerChoreEnabledRequest request) - throws ServiceException { - return stub.isCleanerChoreEnabled(controller, request); - } - - @Override - public ClientProtos.CoprocessorServiceResponse execMasterService( - RpcController controller, ClientProtos.CoprocessorServiceRequest request) - throws ServiceException { - return stub.execMasterService(controller, request); - } - - @Override - public MasterProtos.SnapshotResponse snapshot(RpcController controller, - MasterProtos.SnapshotRequest request) throws ServiceException { - return stub.snapshot(controller, request); - } - - @Override - public MasterProtos.GetCompletedSnapshotsResponse getCompletedSnapshots( - RpcController controller, MasterProtos.GetCompletedSnapshotsRequest request) - throws ServiceException { - return stub.getCompletedSnapshots(controller, request); - } - - @Override - public MasterProtos.DeleteSnapshotResponse deleteSnapshot(RpcController controller, - MasterProtos.DeleteSnapshotRequest request) throws ServiceException { - return stub.deleteSnapshot(controller, request); - } - - @Override - public MasterProtos.IsSnapshotDoneResponse isSnapshotDone(RpcController controller, - MasterProtos.IsSnapshotDoneRequest request) throws ServiceException { - return stub.isSnapshotDone(controller, request); - } - - @Override - public MasterProtos.RestoreSnapshotResponse restoreSnapshot( - RpcController controller, MasterProtos.RestoreSnapshotRequest request) - throws ServiceException { - return stub.restoreSnapshot(controller, request); - } - - @Override - public MasterProtos.ExecProcedureResponse execProcedure( - RpcController controller, MasterProtos.ExecProcedureRequest request) - throws ServiceException { - return stub.execProcedure(controller, request); - } - - @Override - public MasterProtos.ExecProcedureResponse execProcedureWithRet( - RpcController controller, MasterProtos.ExecProcedureRequest request) - throws ServiceException { - return stub.execProcedureWithRet(controller, request); - } - - @Override - public MasterProtos.IsProcedureDoneResponse isProcedureDone(RpcController controller, - MasterProtos.IsProcedureDoneRequest request) throws ServiceException { - return stub.isProcedureDone(controller, request); - } - - @Override - public MasterProtos.GetProcedureResultResponse getProcedureResult(RpcController controller, - MasterProtos.GetProcedureResultRequest request) throws ServiceException { - return stub.getProcedureResult(controller, request); - } - - @Override - public MasterProtos.IsMasterRunningResponse isMasterRunning( - RpcController controller, MasterProtos.IsMasterRunningRequest request) - throws ServiceException { - return stub.isMasterRunning(controller, request); - } - - @Override - public MasterProtos.ModifyNamespaceResponse modifyNamespace(RpcController controller, - MasterProtos.ModifyNamespaceRequest request) - throws ServiceException { - return stub.modifyNamespace(controller, request); - } - - @Override - public MasterProtos.CreateNamespaceResponse createNamespace( - RpcController controller, - MasterProtos.CreateNamespaceRequest request) throws ServiceException { - return stub.createNamespace(controller, request); - } - - @Override - public MasterProtos.DeleteNamespaceResponse deleteNamespace( - RpcController controller, - MasterProtos.DeleteNamespaceRequest request) throws ServiceException { - return stub.deleteNamespace(controller, request); - } - - @Override - public MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor( - RpcController controller, - MasterProtos.GetNamespaceDescriptorRequest request) throws ServiceException { - return stub.getNamespaceDescriptor(controller, request); - } - - @Override - public MasterProtos.ListNamespaceDescriptorsResponse listNamespaceDescriptors( - RpcController controller, - MasterProtos.ListNamespaceDescriptorsRequest request) throws ServiceException { - return stub.listNamespaceDescriptors(controller, request); - } - - @Override - public MasterProtos.ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace( - RpcController controller, MasterProtos.ListTableDescriptorsByNamespaceRequest request) - throws ServiceException { - return stub.listTableDescriptorsByNamespace(controller, request); - } - - @Override - public MasterProtos.ListTableNamesByNamespaceResponse listTableNamesByNamespace( - RpcController controller, MasterProtos.ListTableNamesByNamespaceRequest request) - throws ServiceException { - return stub.listTableNamesByNamespace(controller, request); - } - - @Override - public MasterProtos.GetTableStateResponse getTableState( - RpcController controller, MasterProtos.GetTableStateRequest request) - throws ServiceException { - return stub.getTableState(controller, request); - } - - @Override - public void close() { - release(this.mss); - } - - @Override - public MasterProtos.GetSchemaAlterStatusResponse getSchemaAlterStatus( - RpcController controller, MasterProtos.GetSchemaAlterStatusRequest request) - throws ServiceException { - return stub.getSchemaAlterStatus(controller, request); - } - - @Override - public MasterProtos.GetTableDescriptorsResponse getTableDescriptors( - RpcController controller, MasterProtos.GetTableDescriptorsRequest request) - throws ServiceException { - return stub.getTableDescriptors(controller, request); - } - - @Override - public MasterProtos.GetTableNamesResponse getTableNames( - RpcController controller, MasterProtos.GetTableNamesRequest request) - throws ServiceException { - return stub.getTableNames(controller, request); - } - - @Override - public MasterProtos.GetClusterStatusResponse getClusterStatus( - RpcController controller, MasterProtos.GetClusterStatusRequest request) - throws ServiceException { - return stub.getClusterStatus(controller, request); - } - - @Override - public MasterProtos.SetQuotaResponse setQuota( - RpcController controller, MasterProtos.SetQuotaRequest request) - throws ServiceException { - return stub.setQuota(controller, request); - } - - @Override - public MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestamp( - RpcController controller, MasterProtos.MajorCompactionTimestampRequest request) - throws ServiceException { - return stub.getLastMajorCompactionTimestamp(controller, request); - } - - @Override - public MasterProtos.MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion( - RpcController controller, MasterProtos.MajorCompactionTimestampForRegionRequest request) - throws ServiceException { - return stub.getLastMajorCompactionTimestampForRegion(controller, request); - } - - @Override - public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, - IsBalancerEnabledRequest request) throws ServiceException { - return stub.isBalancerEnabled(controller, request); - } - - @Override - public MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled( - RpcController controller, MasterProtos.SetSplitOrMergeEnabledRequest request) - throws ServiceException { - return stub.setSplitOrMergeEnabled(controller, request); - } - - @Override - public MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled( - RpcController controller, MasterProtos.IsSplitOrMergeEnabledRequest request) - throws ServiceException { - return stub.isSplitOrMergeEnabled(controller, request); - } - - @Override - public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller, - IsNormalizerEnabledRequest request) throws ServiceException { - return stub.isNormalizerEnabled(controller, request); - } - - @Override - public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller, - SecurityCapabilitiesRequest request) throws ServiceException { - return stub.getSecurityCapabilities(controller, request); - } - - @Override - public AddReplicationPeerResponse addReplicationPeer(RpcController controller, - AddReplicationPeerRequest request) throws ServiceException { - return stub.addReplicationPeer(controller, request); - } - - @Override - public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller, - RemoveReplicationPeerRequest request) throws ServiceException { - return stub.removeReplicationPeer(controller, request); - } - - @Override - public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller, - EnableReplicationPeerRequest request) throws ServiceException { - return stub.enableReplicationPeer(controller, request); - } - - @Override - public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller, - DisableReplicationPeerRequest request) throws ServiceException { - return stub.disableReplicationPeer(controller, request); - } - - @Override - public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(RpcController controller, - ListDecommissionedRegionServersRequest request) throws ServiceException { - return stub.listDecommissionedRegionServers(controller, request); - } - - @Override - public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller, - DecommissionRegionServersRequest request) throws ServiceException { - return stub.decommissionRegionServers(controller, request); - } - - @Override - public RecommissionRegionServerResponse recommissionRegionServer( - RpcController controller, RecommissionRegionServerRequest request) - throws ServiceException { - return stub.recommissionRegionServer(controller, request); - } - - @Override - public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller, - GetReplicationPeerConfigRequest request) throws ServiceException { - return stub.getReplicationPeerConfig(controller, request); - } - - @Override - public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig( - RpcController controller, UpdateReplicationPeerConfigRequest request) - throws ServiceException { - return stub.updateReplicationPeerConfig(controller, request); - } - - @Override - public ListReplicationPeersResponse listReplicationPeers(RpcController controller, - ListReplicationPeersRequest request) throws ServiceException { - return stub.listReplicationPeers(controller, request); - } - - @Override - public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes( - RpcController controller, GetSpaceQuotaRegionSizesRequest request) - throws ServiceException { - return stub.getSpaceQuotaRegionSizes(controller, request); - } - - @Override - public GetQuotaStatesResponse getQuotaStates( - RpcController controller, GetQuotaStatesRequest request) throws ServiceException { - return stub.getQuotaStates(controller, request); - } - - @Override - public MasterProtos.ClearDeadServersResponse clearDeadServers(RpcController controller, - MasterProtos.ClearDeadServersRequest request) throws ServiceException { - return stub.clearDeadServers(controller, request); - } - - @Override - public TransitReplicationPeerSyncReplicationStateResponse - transitReplicationPeerSyncReplicationState(RpcController controller, - TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException { - return stub.transitReplicationPeerSyncReplicationState(controller, request); - } - - @Override - public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller, - SwitchRpcThrottleRequest request) throws ServiceException { - return stub.switchRpcThrottle(controller, request); - } - - @Override - public IsRpcThrottleEnabledResponse isRpcThrottleEnabled(RpcController controller, - IsRpcThrottleEnabledRequest request) throws ServiceException { - return stub.isRpcThrottleEnabled(controller, request); - } - - @Override - public SwitchExceedThrottleQuotaResponse switchExceedThrottleQuota(RpcController controller, - SwitchExceedThrottleQuotaRequest request) throws ServiceException { - return stub.switchExceedThrottleQuota(controller, request); - } - - @Override - public AccessControlProtos.GrantResponse grant(RpcController controller, - AccessControlProtos.GrantRequest request) throws ServiceException { - return stub.grant(controller, request); - } - - @Override - public AccessControlProtos.RevokeResponse revoke(RpcController controller, - AccessControlProtos.RevokeRequest request) throws ServiceException { - return stub.revoke(controller, request); - } - - @Override - public GetUserPermissionsResponse getUserPermissions(RpcController controller, - GetUserPermissionsRequest request) throws ServiceException { - return stub.getUserPermissions(controller, request); - } - - @Override - public HasUserPermissionsResponse hasUserPermissions(RpcController controller, - HasUserPermissionsRequest request) throws ServiceException { - return stub.hasUserPermissions(controller, request); - } - }; - } - - private static void release(MasterServiceState mss) { - if (mss != null && mss.connection != null) { - ((ConnectionImplementation)mss.connection).releaseMaster(mss); - } - } - - private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) { - if (mss.getStub() == null){ - return false; - } - try { - return mss.isMasterRunning(); - } catch (UndeclaredThrowableException e) { - // It's somehow messy, but we can receive exceptions such as - // java.net.ConnectException but they're not declared. So we catch it... - LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable()); - return false; - } catch (IOException se) { - LOG.warn("Checking master connection", se); - return false; - } - } - - void releaseMaster(MasterServiceState mss) { - if (mss.getStub() == null) { - return; - } - synchronized (masterLock) { - --mss.userCount; - } - } - - private void closeMasterService(MasterServiceState mss) { - if (mss.getStub() != null) { - LOG.info("Closing master protocol: " + mss); - mss.clearStub(); - } - mss.userCount = 0; - } - - /** - * Immediate close of the shared master. Can be by the delayed close or when closing the - * connection itself. - */ - private void closeMaster() { - synchronized (masterLock) { - closeMasterService(masterServiceState); - } - } - - void updateCachedLocation(RegionInfo hri, ServerName source, ServerName serverName, long seqNum) { - HRegionLocation newHrl = new HRegionLocation(hri, serverName, seqNum); - cacheLocation(hri.getTable(), source, newHrl); - } - - /** - * Deletes cached locations for the specific region. - * @param location The location object for the region, to be purged from cache. - */ - void deleteCachedRegionLocation(final HRegionLocation location) { - metaCache.clearCache(location); - } - - /** - * Update the location with the new value (if the exception is a RegionMovedException) - * or delete it from the cache. Does nothing if we can be sure from the exception that - * the location is still accurate, or if the cache has already been updated. - * @param exception an object (to simplify user code) on which we will try to find a nested - * or wrapped or both RegionMovedException - * @param source server that is the source of the location update. - */ - void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey, - final Object exception, final ServerName source) { - if (rowkey == null || tableName == null) { - LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) + - ", tableName=" + (tableName == null ? "null" : tableName)); - return; - } - - if (source == null) { - // This should not happen, but let's secure ourselves. - return; - } - - if (regionName == null) { - // we do not know which region, so just remove the cache entry for the row and server - if (metrics != null) { - metrics.incrCacheDroppingExceptions(exception); - } - metaCache.clearCache(tableName, rowkey, source); - return; - } - - // Is it something we have already updated? - final RegionLocations oldLocations = getCachedLocation(tableName, rowkey); - HRegionLocation oldLocation = null; - if (oldLocations != null) { - oldLocation = oldLocations.getRegionLocationByRegionName(regionName); - } - if (oldLocation == null || !source.equals(oldLocation.getServerName())) { - // There is no such location in the cache (it's been removed already) or - // the cache has already been refreshed with a different location. => nothing to do - return; - } - - RegionInfo regionInfo = oldLocation.getRegion(); - Throwable cause = ClientExceptionsUtil.findException(exception); - if (cause != null) { - if (!ClientExceptionsUtil.isMetaClearingException(cause)) { - // We know that the region is still on this region server - return; - } - - if (cause instanceof RegionMovedException) { - RegionMovedException rme = (RegionMovedException) cause; - if (LOG.isTraceEnabled()) { - LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " + - rme.getHostname() + ":" + rme.getPort() + - " according to " + source.getAddress()); - } - // We know that the region is not anymore on this region server, but we know - // the new location. - updateCachedLocation( - regionInfo, source, rme.getServerName(), rme.getLocationSeqNum()); - return; - } - } - - if (metrics != null) { - metrics.incrCacheDroppingExceptions(exception); - } - - // If we're here, it means that can cannot be sure about the location, so we remove it from - // the cache. Do not send the source because source can be a new server in the same host:port - metaCache.clearCache(regionInfo); - } - - /** - * @return Default AsyncProcess associated with this connection. - */ - public AsyncProcess getAsyncProcess() { - return asyncProcess; - } - - /** - * @return the current statistics tracker associated with this connection - */ - public ServerStatisticTracker getStatisticsTracker() { - return this.stats; - } - - /** - * @return the configured client backoff policy - */ - public ClientBackoffPolicy getBackoffPolicy() { - return this.backoffPolicy; - } - - /* - * Return the number of cached region for a table. It will only be called - * from a unit test. - */ - @VisibleForTesting - int getNumberOfCachedRegionLocations(final TableName tableName) { - return metaCache.getNumberOfCachedRegionLocations(tableName); - } - - @Override - public void abort(final String msg, Throwable t) { - if (t != null) { - LOG.error(HBaseMarkers.FATAL, msg, t); - } else { - LOG.error(HBaseMarkers.FATAL, msg); - } - this.aborted = true; - close(); - this.closed = true; - } - - @Override - public boolean isClosed() { - return this.closed; - } - - @Override - public boolean isAborted(){ - return this.aborted; - } - - /** - * @return the number of region servers that are currently running - * @throws IOException if a remote or network exception occurs - */ - public int getCurrentNrHRS() throws IOException { - return get(this.registry.getCurrentNrHRS()); - } - - @Override - public void close() { - if (this.closed) { - return; - } - closeMaster(); - shutdownPools(); - if (this.metrics != null) { - this.metrics.shutdown(); - } - this.closed = true; - registry.close(); - this.stubs.clear(); - if (clusterStatusListener != null) { - clusterStatusListener.close(); - } - if (rpcClient != null) { - rpcClient.close(); - } - if (authService != null) { - authService.shutdown(); - } - } - - /** - * Close the connection for good. On the off chance that someone is unable to close - * the connection, perhaps because it bailed out prematurely, the method - * below will ensure that this instance is cleaned up. - * Caveat: The JVM may take an unknown amount of time to call finalize on an - * unreachable object, so our hope is that every consumer cleans up after - * itself, like any good citizen. - */ - @Override - protected void finalize() throws Throwable { - super.finalize(); - close(); - } - - /** - * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration. - */ - public NonceGenerator getNonceGenerator() { - return nonceGenerator; - } - - /** - * Retrieve TableState, represent current table state. - * @param tableName table state for - * @return state of the table - */ - public TableState getTableState(TableName tableName) throws IOException { - checkClosed(); - TableState tableState = MetaTableAccessor.getTableState(this, tableName); - if (tableState == null) { - throw new TableNotFoundException(tableName); - } - return tableState; - } - - /** - * Returns a new RpcRetryingCallerFactory from the given {@link Configuration}. - * This RpcRetryingCallerFactory lets the users create {@link RpcRetryingCaller}s which can be - * intercepted with the configured {@link RetryingCallerInterceptor} - * @param conf configuration - * @return RpcRetryingCallerFactory - */ - public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) { - return RpcRetryingCallerFactory - .instantiate(conf, this.interceptor, this.getStatisticsTracker()); - } - - /** - * @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so - * supports cell blocks. - */ - public boolean hasCellBlockSupport() { - return this.rpcClient.hasCellBlockSupport(); - } - - /** - * @return a ConnectionConfiguration object holding parsed configuration values - */ - public ConnectionConfiguration getConnectionConfiguration() { - return this.connectionConfig; - } - - /** - * @return Connection's RpcRetryingCallerFactory instance - */ - public RpcRetryingCallerFactory getRpcRetryingCallerFactory() { - return this.rpcCallerFactory; - } - - /** - * @return Connection's RpcControllerFactory instance - */ - public RpcControllerFactory getRpcControllerFactory() { - return this.rpcControllerFactory; - } - - private static T get(CompletableFuture future) throws IOException { - try { - return future.get(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw (IOException) new InterruptedIOException().initCause(e); - } catch (ExecutionException e) { - Throwable cause = e.getCause(); - Throwables.propagateIfPossible(cause, IOException.class); - throw new IOException(cause); - } - } - - @Override - public AsyncConnection toAsyncConnection() { - throw new UnsupportedOperationException(); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index a6c47b5535e4..2ac3cd2ac98f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.ServerRpcController; -import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.Threads; @@ -62,7 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; @@ -112,16 +110,6 @@ public static long getPauseTime(final long pause, final int tries) { return normalPause + jitter; } - /** - * @param conn The connection for which to replace the generator. - * @param cnm Replaces the nonce generator used, for testing. - * @return old nonce generator. - */ - public static NonceGenerator injectNonceGeneratorForTesting(ConnectionImplementation conn, - NonceGenerator cnm) { - return ConnectionImplementation.injectNonceGeneratorForTesting(conn, cnm); - } - /** * Changes the configuration to set the number of retries needed when using Connection internally, * e.g. for updating catalog tables, etc. Call this method before we create any Connections. @@ -142,31 +130,6 @@ public static void setServerSideHConnectionRetriesConfig(final Configuration c, log.info(sn + " server-side Connection retries=" + retries); } - /** - * Setup the connection class, so that it will not depend on master being online. Used for testing - * @param conf configuration to set - */ - @VisibleForTesting - public static void setupMasterlessConnection(Configuration conf) { - conf.set(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL, MasterlessConnection.class.getName()); - } - - /** - * Some tests shut down the master. But table availability is a master RPC which is performed on - * region re-lookups. - */ - static class MasterlessConnection extends ConnectionImplementation { - MasterlessConnection(Configuration conf, ExecutorService pool, User user) throws IOException { - super(conf, pool, user); - } - - @Override - public boolean isTableDisabled(TableName tableName) throws IOException { - // treat all tables as enabled - return false; - } - } - /** * Return retires + 1. The returned value will be in range [1, Integer.MAX_VALUE]. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayingRunner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayingRunner.java deleted file mode 100644 index 8ab5d850d2d9..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/DelayingRunner.java +++ /dev/null @@ -1,116 +0,0 @@ -/** - * 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.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; - -import java.util.List; -import java.util.Map; - -/** - * A wrapper for a runnable for a group of actions for a single regionserver. - *

- * This can be used to build up the actions that should be taken and then - *

- *

- * This class exists to simulate using a ScheduledExecutorService with just a regular - * ExecutorService and Runnables. It is used for legacy reasons in the the client; this could - * only be removed if we change the expectations in HTable around the pool the client is able to - * pass in and even if we deprecate the current APIs would require keeping this class around - * for the interim to bridge between the legacy ExecutorServices and the scheduled pool. - *

- */ -@InterfaceAudience.Private -public class DelayingRunner implements Runnable { - private static final Logger LOG = LoggerFactory.getLogger(DelayingRunner.class); - - private final Object sleepLock = new Object(); - private boolean triggerWake = false; - private long sleepTime; - private MultiAction actions = new MultiAction(); - private Runnable runnable; - - public DelayingRunner(long sleepTime, Map.Entry> e) { - this.sleepTime = sleepTime; - add(e); - } - - public void setRunner(Runnable runner) { - this.runnable = runner; - } - - @Override - public void run() { - if (!sleep()) { - LOG.warn( - "Interrupted while sleeping for expected sleep time " + sleepTime + " ms"); - } - //TODO maybe we should consider switching to a listenableFuture for the actual callable and - // then handling the results/errors as callbacks. That way we can decrement outstanding tasks - // even if we get interrupted here, but for now, we still need to run so we decrement the - // outstanding tasks - this.runnable.run(); - } - - /** - * Sleep for an expected amount of time. - *

- * This is nearly a copy of what the Sleeper does, but with the ability to know if you - * got interrupted while sleeping. - *

- * - * @return true if the sleep completely entirely successfully, - * but otherwise false if the sleep was interrupted. - */ - private boolean sleep() { - long now = EnvironmentEdgeManager.currentTime(); - long startTime = now; - long waitTime = sleepTime; - while (waitTime > 0) { - long woke = -1; - try { - synchronized (sleepLock) { - if (triggerWake) break; - sleepLock.wait(waitTime); - } - woke = EnvironmentEdgeManager.currentTime(); - } catch (InterruptedException iex) { - return false; - } - // Recalculate waitTime. - woke = (woke == -1) ? EnvironmentEdgeManager.currentTime() : woke; - waitTime = waitTime - (woke - startTime); - } - return true; - } - - public void add(Map.Entry> e) { - actions.add(e.getKey(), e.getValue()); - } - - public MultiAction getActions() { - return actions; - } - - public long getSleepTime() { - return sleepTime; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java deleted file mode 100644 index 6b0e79096fc9..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FastFailInterceptorContext.java +++ /dev/null @@ -1,135 +0,0 @@ -/** - * 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.commons.lang3.mutable.MutableBoolean; -import org.apache.hadoop.hbase.ServerName; -import org.apache.yetus.audience.InterfaceAudience; - -@InterfaceAudience.Private -class FastFailInterceptorContext extends RetryingCallerInterceptorContext { - - // The variable that indicates whether we were able to connect with the server - // in the last run - private MutableBoolean couldNotCommunicateWithServer = new MutableBoolean(false); - - // If set, we guarantee that no modifications went to server - private MutableBoolean guaranteedClientSideOnly = new MutableBoolean(false); - - // The variable which indicates whether this was a retry or the first time - private boolean didTry = false; - - // The failure info that is associated with the machine which we are trying to - // contact as part of this attempt. - private FailureInfo fInfo = null; - - // Variable indicating that the thread that is currently executing the - // operation is in a mode where it would retry instead of failing fast, so - // that we can figure out whether making contact with the server is - // possible or not. - private boolean retryDespiteFastFailMode = false; - - // The server that would be contacted to successfully complete this operation. - private ServerName server; - - // The number of the retry we are currenty doing. - private int tries; - - public MutableBoolean getCouldNotCommunicateWithServer() { - return couldNotCommunicateWithServer; - } - - public MutableBoolean getGuaranteedClientSideOnly() { - return guaranteedClientSideOnly; - } - - public FailureInfo getFailureInfo() { - return fInfo; - } - - public ServerName getServer() { - return server; - } - - public int getTries() { - return tries; - } - - public boolean didTry() { - return didTry; - } - - public boolean isRetryDespiteFastFailMode() { - return retryDespiteFastFailMode; - } - - public void setCouldNotCommunicateWithServer( - MutableBoolean couldNotCommunicateWithServer) { - this.couldNotCommunicateWithServer = couldNotCommunicateWithServer; - } - - public void setGuaranteedClientSideOnly(MutableBoolean guaranteedClientSideOnly) { - this.guaranteedClientSideOnly = guaranteedClientSideOnly; - } - - public void setDidTry(boolean didTry) { - this.didTry = didTry; - } - - public void setFailureInfo(FailureInfo fInfo) { - this.fInfo = fInfo; - } - - public void setRetryDespiteFastFailMode(boolean retryDespiteFastFailMode) { - this.retryDespiteFastFailMode = retryDespiteFastFailMode; - } - - public void setServer(ServerName server) { - this.server = server; - } - - public void setTries(int tries) { - this.tries = tries; - } - - @Override - public void clear() { - server = null; - fInfo = null; - didTry = false; - couldNotCommunicateWithServer.setValue(false); - guaranteedClientSideOnly.setValue(false); - retryDespiteFastFailMode = false; - tries = 0; - } - - @Override - public FastFailInterceptorContext prepare(RetryingCallable callable) { - return prepare(callable, 0); - } - - @Override - public FastFailInterceptorContext prepare(RetryingCallable callable, int tries) { - if (callable instanceof RegionServerCallable) { - RegionServerCallable retryingCallable = (RegionServerCallable) callable; - server = retryingCallable.getLocation().getServerName(); - } - this.tries = tries; - return this; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java deleted file mode 100644 index d881fe08c6af..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java +++ /dev/null @@ -1,88 +0,0 @@ -/** - * 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 org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse; - -/** - * A Callable for flushRegion() RPC. - */ -@InterfaceAudience.Private -public class FlushRegionCallable extends RegionAdminServiceCallable { - private static final Logger LOG = LoggerFactory.getLogger(FlushRegionCallable.class); - private final byte[] regionName; - private final boolean writeFlushWalMarker; - private boolean reload; - - public FlushRegionCallable(ConnectionImplementation connection, - RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] regionName, - byte[] regionStartKey, boolean writeFlushWalMarker) { - super(connection, rpcControllerFactory, tableName, regionStartKey); - this.regionName = regionName; - this.writeFlushWalMarker = writeFlushWalMarker; - } - - public FlushRegionCallable(ConnectionImplementation connection, - RpcControllerFactory rpcControllerFactory, RegionInfo regionInfo, - boolean writeFlushWalMarker) { - this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(), - regionInfo.getStartKey(), writeFlushWalMarker); - } - - @Override - public void prepare(boolean reload) throws IOException { - super.prepare(reload); - this.reload = reload; - } - - @Override - protected FlushRegionResponse call(HBaseRpcController controller) throws Exception { - // Check whether we should still do the flush to this region. If the regions are changed due - // to splits or merges, etc return success - if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) { - if (!reload) { - throw new IOException("Cached location seems to be different than requested region."); - } - LOG.info("Skipping flush region, because the located region " - + Bytes.toStringBinary(location.getRegionInfo().getRegionName()) + " is different than " - + " requested region " + Bytes.toStringBinary(regionName)); - return FlushRegionResponse.newBuilder() - .setLastFlushTime(EnvironmentEdgeManager.currentTime()) - .setFlushed(false) - .setWroteFlushWalMarker(false) - .build(); - } - - FlushRegionRequest request = - RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker); - return stub.flushRegion(controller, request); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java deleted file mode 100644 index 9c16d05108ac..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ /dev/null @@ -1,3897 +0,0 @@ -/** - * 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.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; -import java.io.Closeable; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CacheEvictionStats; -import org.apache.hadoop.hbase.CacheEvictionStatsBuilder; -import org.apache.hadoop.hbase.ClusterMetrics; -import org.apache.hadoop.hbase.ClusterMetrics.Option; -import org.apache.hadoop.hbase.ClusterMetricsBuilder; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.NamespaceNotFoundException; -import org.apache.hadoop.hbase.NotServingRegionException; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.RegionMetrics; -import org.apache.hadoop.hbase.RegionMetricsBuilder; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; -import org.apache.hadoop.hbase.client.replication.TableCFs; -import org.apache.hadoop.hbase.client.security.SecurityCapability; -import org.apache.hadoop.hbase.exceptions.TimeoutIOException; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.quotas.QuotaFilter; -import org.apache.hadoop.hbase.quotas.QuotaRetriever; -import org.apache.hadoop.hbase.quotas.QuotaSettings; -import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot; -import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; -import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; -import org.apache.hadoop.hbase.replication.SyncReplicationState; -import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest; -import org.apache.hadoop.hbase.security.access.Permission; -import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil; -import org.apache.hadoop.hbase.security.access.UserPermission; -import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; -import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; -import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; -import org.apache.hadoop.hbase.util.Addressing; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ForeignExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.util.StringUtils; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearRegionBlockCacheResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.StopServerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateConfigurationRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; - -/** - * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that - * this is an HBase-internal class as defined in - * https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/InterfaceClassification.html - * There are no guarantees for backwards source / binary compatibility and methods or class can - * change or go away without deprecation. - * Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead of constructing - * an HBaseAdmin directly. - * - *

Connection should be an unmanaged connection obtained via - * {@link ConnectionFactory#createConnection(Configuration)} - * - * @see ConnectionFactory - * @see Connection - * @see Admin - */ -@InterfaceAudience.Private -public class HBaseAdmin implements Admin { - private static final Logger LOG = LoggerFactory.getLogger(HBaseAdmin.class); - - private ConnectionImplementation connection; - - private final Configuration conf; - private final long pause; - private final int numRetries; - private final int syncWaitTimeout; - private boolean aborted; - private int operationTimeout; - private int rpcTimeout; - private int getProcedureTimeout; - - private RpcRetryingCallerFactory rpcCallerFactory; - private RpcControllerFactory rpcControllerFactory; - - private NonceGenerator ng; - - @Override - public int getOperationTimeout() { - return operationTimeout; - } - - @Override - public int getSyncWaitTimeout() { - return syncWaitTimeout; - } - - HBaseAdmin(ConnectionImplementation connection) throws IOException { - this.conf = connection.getConfiguration(); - this.connection = connection; - - // TODO: receive ConnectionConfiguration here rather than re-parsing these configs every time. - this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, - HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); - this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - this.rpcTimeout = this.conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - this.syncWaitTimeout = this.conf.getInt( - "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min - this.getProcedureTimeout = - this.conf.getInt("hbase.client.procedure.future.get.timeout.msec", 10 * 60000); // 10min - - this.rpcCallerFactory = connection.getRpcRetryingCallerFactory(); - this.rpcControllerFactory = connection.getRpcControllerFactory(); - - this.ng = this.connection.getNonceGenerator(); - } - - @Override - public void abort(String why, Throwable e) { - // Currently does nothing but throw the passed message and exception - this.aborted = true; - throw new RuntimeException(why, e); - } - - @Override - public boolean isAborted() { - return this.aborted; - } - - @Override - public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning) - throws IOException { - return get(abortProcedureAsync(procId, mayInterruptIfRunning), this.syncWaitTimeout, - TimeUnit.MILLISECONDS); - } - - @Override - public Future abortProcedureAsync(final long procId, final boolean mayInterruptIfRunning) - throws IOException { - Boolean abortProcResponse = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected AbortProcedureResponse rpcCall() throws Exception { - AbortProcedureRequest abortProcRequest = - AbortProcedureRequest.newBuilder().setProcId(procId).build(); - return master.abortProcedure(getRpcController(), abortProcRequest); - } - }).getIsProcedureAborted(); - return new AbortProcedureFuture(this, procId, abortProcResponse); - } - - @Override - public List listTableDescriptors() throws IOException { - return listTableDescriptors((Pattern)null, false); - } - - @Override - public List listTableDescriptors(Pattern pattern, boolean includeSysTables) - throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables); - return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), - req)); - } - }); - } - - @Override - public TableDescriptor getDescriptor(TableName tableName) - throws TableNotFoundException, IOException { - return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory, - operationTimeout, rpcTimeout); - } - - @Override - public Future modifyTableAsync(TableDescriptor td) throws IOException { - ModifyTableResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected ModifyTableResponse rpcCall() throws Exception { - setPriority(td.getTableName()); - ModifyTableRequest request = RequestConverter.buildModifyTableRequest( - td.getTableName(), td, nonceGroup, nonce); - return master.modifyTable(getRpcController(), request); - } - }); - return new ModifyTableFuture(this, td.getTableName(), response); - } - - @Override - public List listTableDescriptorsByNamespace(byte[] name) throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - return master.listTableDescriptorsByNamespace(getRpcController(), - ListTableDescriptorsByNamespaceRequest.newBuilder() - .setNamespaceName(Bytes.toString(name)).build()) - .getTableSchemaList() - .stream() - .map(ProtobufUtil::toTableDescriptor) - .collect(Collectors.toList()); - } - }); - } - - @Override - public List listTableDescriptors(List tableNames) throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest(tableNames); - return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), - req)); - } - }); - } - - @Override - public List getRegions(final ServerName sn) throws IOException { - AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - return ProtobufUtil.getOnlineRegions(controller, admin); - } - - @Override - public List getRegions(TableName tableName) throws IOException { - if (TableName.isMetaTableName(tableName)) { - return Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO); - } else { - return MetaTableAccessor.getTableRegions(connection, tableName, true); - } - } - - private static class AbortProcedureFuture extends ProcedureFuture { - private boolean isAbortInProgress; - - public AbortProcedureFuture( - final HBaseAdmin admin, - final Long procId, - final Boolean abortProcResponse) { - super(admin, procId); - this.isAbortInProgress = abortProcResponse; - } - - @Override - public Boolean get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - if (!this.isAbortInProgress) { - return false; - } - super.get(timeout, unit); - return true; - } - } - - /** @return Connection used by this object. */ - @Override - public ConnectionImplementation getConnection() { - return connection; - } - - @Override - public boolean tableExists(final TableName tableName) throws IOException { - return executeCallable(new RpcRetryingCallable() { - @Override - protected Boolean rpcCall(int callTimeout) throws Exception { - return MetaTableAccessor.tableExists(connection, tableName); - } - }); - } - - @Override - public TableName[] listTableNames() throws IOException { - return listTableNames((Pattern)null, false); - } - - @Override - public TableName[] listTableNames(Pattern pattern) throws IOException { - return listTableNames(pattern, false); - } - - @Override - public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables) - throws IOException { - return executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected TableName[] rpcCall() throws Exception { - GetTableNamesRequest req = - RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables); - return ProtobufUtil.getTableNameArray(master.getTableNames(getRpcController(), req) - .getTableNamesList()); - } - }); - } - - static TableDescriptor getTableDescriptor(final TableName tableName, - ConnectionImplementation connection, RpcRetryingCallerFactory rpcCallerFactory, - final RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) - throws IOException { - if (tableName == null) return null; - TableDescriptor td = - executeCallable(new MasterCallable(connection, rpcControllerFactory) { - @Override - protected TableDescriptor rpcCall() throws Exception { - GetTableDescriptorsRequest req = - RequestConverter.buildGetTableDescriptorsRequest(tableName); - GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req); - if (!htds.getTableSchemaList().isEmpty()) { - return ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)); - } - return null; - } - }, rpcCallerFactory, operationTimeout, rpcTimeout); - if (td != null) { - return td; - } - throw new TableNotFoundException(tableName.getNameAsString()); - } - - private long getPauseTime(int tries) { - int triesCount = tries; - if (triesCount >= HConstants.RETRY_BACKOFF.length) { - triesCount = HConstants.RETRY_BACKOFF.length - 1; - } - return this.pause * HConstants.RETRY_BACKOFF[triesCount]; - } - - @Override - public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) - throws IOException { - if (numRegions < 3) { - throw new IllegalArgumentException("Must create at least three regions"); - } else if (Bytes.compareTo(startKey, endKey) >= 0) { - throw new IllegalArgumentException("Start key must be smaller than end key"); - } - if (numRegions == 3) { - createTable(desc, new byte[][] { startKey, endKey }); - return; - } - byte[][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3); - if (splitKeys == null || splitKeys.length != numRegions - 1) { - throw new IllegalArgumentException("Unable to split key range into enough regions"); - } - createTable(desc, splitKeys); - } - - @Override - public Future createTableAsync(final TableDescriptor desc, final byte[][] splitKeys) - throws IOException { - if (desc.getTableName() == null) { - throw new IllegalArgumentException("TableName cannot be null"); - } - if (splitKeys != null && splitKeys.length > 0) { - Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR); - // Verify there are no duplicate split keys - byte[] lastKey = null; - for (byte[] splitKey : splitKeys) { - if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) { - throw new IllegalArgumentException( - "Empty split key must not be passed in the split keys."); - } - if (lastKey != null && Bytes.equals(splitKey, lastKey)) { - throw new IllegalArgumentException("All split keys must be unique, " + - "found duplicate: " + Bytes.toStringBinary(splitKey) + - ", " + Bytes.toStringBinary(lastKey)); - } - lastKey = splitKey; - } - } - - CreateTableResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected CreateTableResponse rpcCall() throws Exception { - setPriority(desc.getTableName()); - CreateTableRequest request = RequestConverter.buildCreateTableRequest( - desc, splitKeys, nonceGroup, nonce); - return master.createTable(getRpcController(), request); - } - }); - return new CreateTableFuture(this, desc, splitKeys, response); - } - - private static class CreateTableFuture extends TableFuture { - private final TableDescriptor desc; - private final byte[][] splitKeys; - - public CreateTableFuture(final HBaseAdmin admin, final TableDescriptor desc, - final byte[][] splitKeys, final CreateTableResponse response) { - super(admin, desc.getTableName(), - (response != null && response.hasProcId()) ? response.getProcId() : null); - this.splitKeys = splitKeys; - this.desc = desc; - } - - @Override - protected TableDescriptor getDescriptor() { - return desc; - } - - @Override - public String getOperationType() { - return "CREATE"; - } - - @Override - protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException { - waitForTableEnabled(deadlineTs); - waitForAllRegionsOnline(deadlineTs, splitKeys); - return null; - } - } - - @Override - public Future deleteTableAsync(final TableName tableName) throws IOException { - DeleteTableResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected DeleteTableResponse rpcCall() throws Exception { - setPriority(tableName); - DeleteTableRequest req = - RequestConverter.buildDeleteTableRequest(tableName, nonceGroup,nonce); - return master.deleteTable(getRpcController(), req); - } - }); - return new DeleteTableFuture(this, tableName, response); - } - - private static class DeleteTableFuture extends TableFuture { - public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName, - final DeleteTableResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - } - - @Override - public String getOperationType() { - return "DELETE"; - } - - @Override - protected Void waitOperationResult(final long deadlineTs) - throws IOException, TimeoutException { - waitTableNotFound(deadlineTs); - return null; - } - - @Override - protected Void postOperationResult(final Void result, final long deadlineTs) - throws IOException, TimeoutException { - // Delete cached information to prevent clients from using old locations - try (RegionLocator locator = getAdmin().getConnection().getRegionLocator(getTableName())) { - locator.clearRegionLocationCache(); - } - return super.postOperationResult(result, deadlineTs); - } - } - - @Override - public Future truncateTableAsync(final TableName tableName, final boolean preserveSplits) - throws IOException { - TruncateTableResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected TruncateTableResponse rpcCall() throws Exception { - setPriority(tableName); - LOG.info("Started truncating " + tableName); - TruncateTableRequest req = RequestConverter.buildTruncateTableRequest( - tableName, preserveSplits, nonceGroup, nonce); - return master.truncateTable(getRpcController(), req); - } - }); - return new TruncateTableFuture(this, tableName, preserveSplits, response); - } - - private static class TruncateTableFuture extends TableFuture { - private final boolean preserveSplits; - - public TruncateTableFuture(final HBaseAdmin admin, final TableName tableName, - final boolean preserveSplits, final TruncateTableResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - this.preserveSplits = preserveSplits; - } - - @Override - public String getOperationType() { - return "TRUNCATE"; - } - - @Override - protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException { - waitForTableEnabled(deadlineTs); - // once the table is enabled, we know the operation is done. so we can fetch the splitKeys - byte[][] splitKeys = preserveSplits ? getAdmin().getTableSplits(getTableName()) : null; - waitForAllRegionsOnline(deadlineTs, splitKeys); - return null; - } - } - - private byte[][] getTableSplits(final TableName tableName) throws IOException { - byte[][] splits = null; - try (RegionLocator locator = getConnection().getRegionLocator(tableName)) { - byte[][] startKeys = locator.getStartKeys(); - if (startKeys.length == 1) { - return splits; - } - splits = new byte[startKeys.length - 1][]; - for (int i = 1; i < startKeys.length; i++) { - splits[i - 1] = startKeys[i]; - } - } - return splits; - } - - @Override - public Future enableTableAsync(final TableName tableName) throws IOException { - TableName.isLegalFullyQualifiedTableName(tableName.getName()); - EnableTableResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected EnableTableResponse rpcCall() throws Exception { - setPriority(tableName); - LOG.info("Started enable of " + tableName); - EnableTableRequest req = - RequestConverter.buildEnableTableRequest(tableName, nonceGroup, nonce); - return master.enableTable(getRpcController(),req); - } - }); - return new EnableTableFuture(this, tableName, response); - } - - private static class EnableTableFuture extends TableFuture { - public EnableTableFuture(final HBaseAdmin admin, final TableName tableName, - final EnableTableResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - } - - @Override - public String getOperationType() { - return "ENABLE"; - } - - @Override - protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException { - waitForTableEnabled(deadlineTs); - return null; - } - } - - @Override - public Future disableTableAsync(final TableName tableName) throws IOException { - TableName.isLegalFullyQualifiedTableName(tableName.getName()); - DisableTableResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected DisableTableResponse rpcCall() throws Exception { - setPriority(tableName); - LOG.info("Started disable of " + tableName); - DisableTableRequest req = - RequestConverter.buildDisableTableRequest( - tableName, nonceGroup, nonce); - return master.disableTable(getRpcController(), req); - } - }); - return new DisableTableFuture(this, tableName, response); - } - - private static class DisableTableFuture extends TableFuture { - public DisableTableFuture(final HBaseAdmin admin, final TableName tableName, - final DisableTableResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - } - - @Override - public String getOperationType() { - return "DISABLE"; - } - - @Override - protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException { - waitForTableDisabled(deadlineTs); - return null; - } - } - - @Override - public boolean isTableEnabled(final TableName tableName) throws IOException { - checkTableExists(tableName); - return executeCallable(new RpcRetryingCallable() { - @Override - protected Boolean rpcCall(int callTimeout) throws Exception { - TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName); - if (tableState == null) { - throw new TableNotFoundException(tableName); - } - return tableState.inStates(TableState.State.ENABLED); - } - }); - } - - @Override - public boolean isTableDisabled(TableName tableName) throws IOException { - checkTableExists(tableName); - return connection.isTableDisabled(tableName); - } - - @Override - public boolean isTableAvailable(TableName tableName) throws IOException { - return connection.isTableAvailable(tableName, null); - } - - @Override - public Future addColumnFamilyAsync(final TableName tableName, - final ColumnFamilyDescriptor columnFamily) throws IOException { - AddColumnResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected AddColumnResponse rpcCall() throws Exception { - setPriority(tableName); - AddColumnRequest req = - RequestConverter.buildAddColumnRequest(tableName, columnFamily, nonceGroup, nonce); - return master.addColumn(getRpcController(), req); - } - }); - return new AddColumnFamilyFuture(this, tableName, response); - } - - private static class AddColumnFamilyFuture extends ModifyTableFuture { - public AddColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName, - final AddColumnResponse response) { - super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId() - : null); - } - - @Override - public String getOperationType() { - return "ADD_COLUMN_FAMILY"; - } - } - - @Override - public Future deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily) - throws IOException { - DeleteColumnResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected DeleteColumnResponse rpcCall() throws Exception { - setPriority(tableName); - DeleteColumnRequest req = - RequestConverter.buildDeleteColumnRequest(tableName, columnFamily, - nonceGroup, nonce); - return master.deleteColumn(getRpcController(), req); - } - }); - return new DeleteColumnFamilyFuture(this, tableName, response); - } - - private static class DeleteColumnFamilyFuture extends ModifyTableFuture { - public DeleteColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName, - final DeleteColumnResponse response) { - super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId() - : null); - } - - @Override - public String getOperationType() { - return "DELETE_COLUMN_FAMILY"; - } - } - - @Override - public Future modifyColumnFamilyAsync(final TableName tableName, - final ColumnFamilyDescriptor columnFamily) throws IOException { - ModifyColumnResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected ModifyColumnResponse rpcCall() throws Exception { - setPriority(tableName); - ModifyColumnRequest req = - RequestConverter.buildModifyColumnRequest(tableName, columnFamily, - nonceGroup, nonce); - return master.modifyColumn(getRpcController(), req); - } - }); - return new ModifyColumnFamilyFuture(this, tableName, response); - } - - private static class ModifyColumnFamilyFuture extends ModifyTableFuture { - public ModifyColumnFamilyFuture(final HBaseAdmin admin, final TableName tableName, - final ModifyColumnResponse response) { - super(admin, tableName, (response != null && response.hasProcId()) ? response.getProcId() - : null); - } - - @Override - public String getOperationType() { - return "MODIFY_COLUMN_FAMILY"; - } - } - - @Override - public void flush(final TableName tableName) throws IOException { - checkTableExists(tableName); - if (isTableDisabled(tableName)) { - LOG.info("Table is disabled: " + tableName.getNameAsString()); - return; - } - execProcedure("flush-table-proc", tableName.getNameAsString(), new HashMap<>()); - } - - @Override - public void flushRegion(final byte[] regionName) throws IOException { - Pair regionServerPair = getRegion(regionName); - if (regionServerPair == null) { - throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName)); - } - if (regionServerPair.getSecond() == null) { - throw new NoServerForRegionException(Bytes.toStringBinary(regionName)); - } - final RegionInfo regionInfo = regionServerPair.getFirst(); - ServerName serverName = regionServerPair.getSecond(); - flush(this.connection.getAdmin(serverName), regionInfo); - } - - private void flush(AdminService.BlockingInterface admin, final RegionInfo info) - throws IOException { - ProtobufUtil.call(() -> { - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - FlushRegionRequest request = - RequestConverter.buildFlushRegionRequest(info.getRegionName()); - admin.flushRegion(controller, request); - return null; - }); - } - - @Override - public void flushRegionServer(ServerName serverName) throws IOException { - for (RegionInfo region : getRegions(serverName)) { - flush(this.connection.getAdmin(serverName), region); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void compact(final TableName tableName) - throws IOException { - compact(tableName, null, false, CompactType.NORMAL); - } - - @Override - public void compactRegion(final byte[] regionName) - throws IOException { - compactRegion(regionName, null, false); - } - - /** - * {@inheritDoc} - */ - @Override - public void compact(final TableName tableName, final byte[] columnFamily) - throws IOException { - compact(tableName, columnFamily, false, CompactType.NORMAL); - } - - /** - * {@inheritDoc} - */ - @Override - public void compactRegion(final byte[] regionName, final byte[] columnFamily) - throws IOException { - compactRegion(regionName, columnFamily, false); - } - - @Override - public Map compactionSwitch(boolean switchState, List - serverNamesList) throws IOException { - List serverList = new ArrayList<>(); - if (serverNamesList.isEmpty()) { - ClusterMetrics status = getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)); - serverList.addAll(status.getLiveServerMetrics().keySet()); - } else { - for (String regionServerName: serverNamesList) { - ServerName serverName = null; - try { - serverName = ServerName.valueOf(regionServerName); - } catch (Exception e) { - throw new IllegalArgumentException(String.format("Invalid ServerName format: %s", - regionServerName)); - } - if (serverName == null) { - throw new IllegalArgumentException(String.format("Null ServerName: %s", - regionServerName)); - } - serverList.add(serverName); - } - } - Map res = new HashMap<>(serverList.size()); - for (ServerName serverName: serverList) { - boolean prev_state = switchCompact(this.connection.getAdmin(serverName), switchState); - res.put(serverName, prev_state); - } - return res; - } - - private Boolean switchCompact(AdminService.BlockingInterface admin, boolean onOrOff) - throws IOException { - return executeCallable(new RpcRetryingCallable() { - @Override protected Boolean rpcCall(int callTimeout) throws Exception { - HBaseRpcController controller = rpcControllerFactory.newController(); - CompactionSwitchRequest request = - CompactionSwitchRequest.newBuilder().setEnabled(onOrOff).build(); - CompactionSwitchResponse compactionSwitchResponse = - admin.compactionSwitch(controller, request); - return compactionSwitchResponse.getPrevState(); - } - }); - } - - @Override - public void compactRegionServer(final ServerName serverName) throws IOException { - for (RegionInfo region : getRegions(serverName)) { - compact(this.connection.getAdmin(serverName), region, false, null); - } - } - - @Override - public void majorCompactRegionServer(final ServerName serverName) throws IOException { - for (RegionInfo region : getRegions(serverName)) { - compact(this.connection.getAdmin(serverName), region, true, null); - } - } - - @Override - public void majorCompact(final TableName tableName) - throws IOException { - compact(tableName, null, true, CompactType.NORMAL); - } - - @Override - public void majorCompactRegion(final byte[] regionName) - throws IOException { - compactRegion(regionName, null, true); - } - - /** - * {@inheritDoc} - */ - @Override - public void majorCompact(final TableName tableName, final byte[] columnFamily) - throws IOException { - compact(tableName, columnFamily, true, CompactType.NORMAL); - } - - @Override - public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily) - throws IOException { - compactRegion(regionName, columnFamily, true); - } - - /** - * Compact a table. - * Asynchronous operation. - * - * @param tableName table or region to compact - * @param columnFamily column family within a table or region - * @param major True if we are to do a major compaction. - * @param compactType {@link org.apache.hadoop.hbase.client.CompactType} - * @throws IOException if a remote or network exception occurs - */ - private void compact(final TableName tableName, final byte[] columnFamily,final boolean major, - CompactType compactType) throws IOException { - switch (compactType) { - case MOB: - compact(this.connection.getAdminForMaster(), RegionInfo.createMobRegionInfo(tableName), - major, columnFamily); - break; - case NORMAL: - checkTableExists(tableName); - for (HRegionLocation loc :connection.locateRegions(tableName, false, false)) { - ServerName sn = loc.getServerName(); - if (sn == null) { - continue; - } - try { - compact(this.connection.getAdmin(sn), loc.getRegion(), major, columnFamily); - } catch (NotServingRegionException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to" + (major ? " major" : "") + " compact " + loc.getRegion() + - ": " + StringUtils.stringifyException(e)); - } - } - } - break; - default: - throw new IllegalArgumentException("Unknown compactType: " + compactType); - } - } - - /** - * Compact an individual region. - * Asynchronous operation. - * - * @param regionName region to compact - * @param columnFamily column family within a table or region - * @param major True if we are to do a major compaction. - * @throws IOException if a remote or network exception occurs - * @throws InterruptedException - */ - private void compactRegion(final byte[] regionName, final byte[] columnFamily, - final boolean major) throws IOException { - Pair regionServerPair = getRegion(regionName); - if (regionServerPair == null) { - throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName)); - } - if (regionServerPair.getSecond() == null) { - throw new NoServerForRegionException(Bytes.toStringBinary(regionName)); - } - compact(this.connection.getAdmin(regionServerPair.getSecond()), regionServerPair.getFirst(), - major, columnFamily); - } - - private void compact(AdminService.BlockingInterface admin, RegionInfo hri, boolean major, - byte[] family) throws IOException { - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - CompactRegionRequest request = - RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family); - admin.compactRegion(controller, request); - return null; - } - }; - ProtobufUtil.call(callable); - } - - @Override - public void move(byte[] encodedRegionName) throws IOException { - move(encodedRegionName, null); - } - - public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - setPriority(encodedRegionName); - MoveRegionRequest request = - RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName); - master.moveRegion(getRpcController(), request); - return null; - } - }); - } - - @Override - public void assign(final byte [] regionName) throws MasterNotRunningException, - ZooKeeperConnectionException, IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - setPriority(regionName); - AssignRegionRequest request = - RequestConverter.buildAssignRegionRequest(getRegionName(regionName)); - master.assignRegion(getRpcController(), request); - return null; - } - }); - } - - @Override - public void unassign(final byte [] regionName, final boolean force) throws IOException { - final byte[] toBeUnassigned = getRegionName(regionName); - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - setPriority(regionName); - UnassignRegionRequest request = - RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force); - master.unassignRegion(getRpcController(), request); - return null; - } - }); - } - - @Override - public void offline(final byte [] regionName) - throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - setPriority(regionName); - master.offlineRegion(getRpcController(), - RequestConverter.buildOfflineRegionRequest(regionName)); - return null; - } - }); - } - - @Override - public boolean balancerSwitch(final boolean on, final boolean synchronous) - throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - SetBalancerRunningRequest req = - RequestConverter.buildSetBalancerRunningRequest(on, synchronous); - return master.setBalancerRunning(getRpcController(), req).getPrevBalanceValue(); - } - }); - } - - @Override - public boolean balance() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.balance(getRpcController(), - RequestConverter.buildBalanceRequest(false)).getBalancerRan(); - } - }); - } - - @Override - public boolean balance(final boolean force) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.balance(getRpcController(), - RequestConverter.buildBalanceRequest(force)).getBalancerRan(); - } - }); - } - - @Override - public boolean isBalancerEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.isBalancerEnabled(getRpcController(), - RequestConverter.buildIsBalancerEnabledRequest()).getEnabled(); - } - }); - } - - /** - * {@inheritDoc} - */ - @Override - public CacheEvictionStats clearBlockCache(final TableName tableName) throws IOException { - checkTableExists(tableName); - CacheEvictionStatsBuilder cacheEvictionStats = CacheEvictionStats.builder(); - List> pairs = - MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); - Map> regionInfoByServerName = - pairs.stream() - .filter(pair -> !(pair.getFirst().isOffline())) - .filter(pair -> pair.getSecond() != null) - .collect(Collectors.groupingBy(pair -> pair.getSecond(), - Collectors.mapping(pair -> pair.getFirst(), Collectors.toList()))); - - for (Map.Entry> entry : regionInfoByServerName.entrySet()) { - CacheEvictionStats stats = clearBlockCache(entry.getKey(), entry.getValue()); - cacheEvictionStats = cacheEvictionStats.append(stats); - if (stats.getExceptionCount() > 0) { - for (Map.Entry exception : stats.getExceptions().entrySet()) { - LOG.debug("Failed to clear block cache for " - + Bytes.toStringBinary(exception.getKey()) - + " on " + entry.getKey() + ": ", exception.getValue()); - } - } - } - return cacheEvictionStats.build(); - } - - private CacheEvictionStats clearBlockCache(final ServerName sn, final List hris) - throws IOException { - HBaseRpcController controller = rpcControllerFactory.newController(); - AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - ClearRegionBlockCacheRequest request = - RequestConverter.buildClearRegionBlockCacheRequest(hris); - ClearRegionBlockCacheResponse response; - try { - response = admin.clearRegionBlockCache(controller, request); - return ProtobufUtil.toCacheEvictionStats(response.getStats()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - - /** - * Invoke region normalizer. Can NOT run for various reasons. Check logs. - * - * @return True if region normalizer ran, false otherwise. - */ - @Override - public boolean normalize() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.normalize(getRpcController(), - RequestConverter.buildNormalizeRequest()).getNormalizerRan(); - } - }); - } - - @Override - public boolean isNormalizerEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.isNormalizerEnabled(getRpcController(), - RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled(); - } - }); - } - - @Override - public boolean normalizerSwitch(final boolean on) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - SetNormalizerRunningRequest req = - RequestConverter.buildSetNormalizerRunningRequest(on); - return master.setNormalizerRunning(getRpcController(), req).getPrevNormalizerValue(); - } - }); - } - - @Override - public boolean catalogJanitorSwitch(final boolean enable) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.enableCatalogJanitor(getRpcController(), - RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue(); - } - }); - } - - @Override - public int runCatalogJanitor() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Integer rpcCall() throws Exception { - return master.runCatalogScan(getRpcController(), - RequestConverter.buildCatalogScanRequest()).getScanResult(); - } - }); - } - - @Override - public boolean isCatalogJanitorEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.isCatalogJanitorEnabled(getRpcController(), - RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue(); - } - }); - } - - @Override - public boolean cleanerChoreSwitch(final boolean on) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override public Boolean rpcCall() throws Exception { - return master.setCleanerChoreRunning(getRpcController(), - RequestConverter.buildSetCleanerChoreRunningRequest(on)).getPrevValue(); - } - }); - } - - @Override - public boolean runCleanerChore() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override public Boolean rpcCall() throws Exception { - return master.runCleanerChore(getRpcController(), - RequestConverter.buildRunCleanerChoreRequest()).getCleanerChoreRan(); - } - }); - } - - @Override - public boolean isCleanerChoreEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override public Boolean rpcCall() throws Exception { - return master.isCleanerChoreEnabled(getRpcController(), - RequestConverter.buildIsCleanerChoreEnabledRequest()).getValue(); - } - }); - } - - /** - * Merge two regions. Synchronous operation. - * Note: It is not feasible to predict the length of merge. - * Therefore, this is for internal testing only. - * @param nameOfRegionA encoded or full name of region a - * @param nameOfRegionB encoded or full name of region b - * @param forcible true if do a compulsory merge, otherwise we will only merge - * two adjacent regions - * @throws IOException - */ - @VisibleForTesting - public void mergeRegionsSync( - final byte[] nameOfRegionA, - final byte[] nameOfRegionB, - final boolean forcible) throws IOException { - get( - mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible), - syncWaitTimeout, - TimeUnit.MILLISECONDS); - } - - /** - * Merge two regions. Asynchronous operation. - * @param nameofRegionsToMerge encoded or full name of daughter regions - * @param forcible true if do a compulsory merge, otherwise we will only merge - * adjacent regions - */ - @Override - public Future mergeRegionsAsync(final byte[][] nameofRegionsToMerge, final boolean forcible) - throws IOException { - Preconditions.checkArgument(nameofRegionsToMerge.length >= 2, "Can not merge only %s region", - nameofRegionsToMerge.length); - byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][]; - for (int i = 0; i < nameofRegionsToMerge.length; i++) { - encodedNameofRegionsToMerge[i] = - RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ? nameofRegionsToMerge[i] - : Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i])); - } - - TableName tableName = null; - Pair pair; - - for(int i = 0; i < nameofRegionsToMerge.length; i++) { - pair = getRegion(nameofRegionsToMerge[i]); - - if (pair != null) { - if (pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { - throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly"); - } - if (tableName == null) { - tableName = pair.getFirst().getTable(); - } else if (!tableName.equals(pair.getFirst().getTable())) { - throw new IllegalArgumentException ("Cannot merge regions from two different tables " + - tableName + " and " + pair.getFirst().getTable()); - } - } else { - throw new UnknownRegionException ( - "Can't invoke merge on unknown region " - + Bytes.toStringBinary(encodedNameofRegionsToMerge[i])); - } - } - - MergeTableRegionsResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected MergeTableRegionsResponse rpcCall() throws Exception { - MergeTableRegionsRequest request = RequestConverter - .buildMergeTableRegionsRequest( - encodedNameofRegionsToMerge, - forcible, - nonceGroup, - nonce); - return master.mergeTableRegions(getRpcController(), request); - } - }); - return new MergeTableRegionsFuture(this, tableName, response); - } - - private static class MergeTableRegionsFuture extends TableFuture { - public MergeTableRegionsFuture( - final HBaseAdmin admin, - final TableName tableName, - final MergeTableRegionsResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - } - - public MergeTableRegionsFuture( - final HBaseAdmin admin, - final TableName tableName, - final Long procId) { - super(admin, tableName, procId); - } - - @Override - public String getOperationType() { - return "MERGE_REGIONS"; - } - } - /** - * Split one region. Synchronous operation. - * Note: It is not feasible to predict the length of split. - * Therefore, this is for internal testing only. - * @param regionName encoded or full name of region - * @param splitPoint key where region splits - * @throws IOException - */ - @VisibleForTesting - public void splitRegionSync(byte[] regionName, byte[] splitPoint) throws IOException { - splitRegionSync(regionName, splitPoint, syncWaitTimeout, TimeUnit.MILLISECONDS); - } - - - /** - * Split one region. Synchronous operation. - * @param regionName region to be split - * @param splitPoint split point - * @param timeout how long to wait on split - * @param units time units - * @throws IOException - */ - public void splitRegionSync(byte[] regionName, byte[] splitPoint, final long timeout, - final TimeUnit units) throws IOException { - get(splitRegionAsync(regionName, splitPoint), timeout, units); - } - - @Override - public Future splitRegionAsync(byte[] regionName, byte[] splitPoint) - throws IOException { - byte[] encodedNameofRegionToSplit = RegionInfo.isEncodedRegionName(regionName) ? - regionName : Bytes.toBytes(RegionInfo.encodeRegionName(regionName)); - Pair pair = getRegion(regionName); - if (pair != null) { - if (pair.getFirst() != null && - pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { - throw new IllegalArgumentException ("Can't invoke split on non-default regions directly"); - } - } else { - throw new UnknownRegionException( - "Can't invoke split on unknown region " + Bytes.toStringBinary(encodedNameofRegionToSplit)); - } - - return splitRegionAsync(pair.getFirst(), splitPoint); - } - - Future splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException { - TableName tableName = hri.getTable(); - if (hri.getStartKey() != null && splitPoint != null && - Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) { - throw new IOException("should not give a splitkey which equals to startkey!"); - } - - SplitTableRegionResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected SplitTableRegionResponse rpcCall() throws Exception { - setPriority(tableName); - SplitTableRegionRequest request = RequestConverter - .buildSplitTableRegionRequest(hri, splitPoint, nonceGroup, nonce); - return master.splitRegion(getRpcController(), request); - } - }); - return new SplitTableRegionFuture(this, tableName, response); - } - - private static class SplitTableRegionFuture extends TableFuture { - public SplitTableRegionFuture(final HBaseAdmin admin, - final TableName tableName, - final SplitTableRegionResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - } - - public SplitTableRegionFuture( - final HBaseAdmin admin, - final TableName tableName, - final Long procId) { - super(admin, tableName, procId); - } - - @Override - public String getOperationType() { - return "SPLIT_REGION"; - } - } - - @Override - public void split(final TableName tableName) throws IOException { - split(tableName, null); - } - - @Override - public void split(final TableName tableName, final byte[] splitPoint) throws IOException { - checkTableExists(tableName); - for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) { - ServerName sn = loc.getServerName(); - if (sn == null) { - continue; - } - RegionInfo r = loc.getRegion(); - // check for parents - if (r.isSplitParent()) { - continue; - } - // if a split point given, only split that particular region - if (r.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID || - (splitPoint != null && !r.containsRow(splitPoint))) { - continue; - } - // call out to master to do split now - splitRegionAsync(r, splitPoint); - } - } - - private static class ModifyTableFuture extends TableFuture { - public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName, - final ModifyTableResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - } - - public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) { - super(admin, tableName, procId); - } - - @Override - public String getOperationType() { - return "MODIFY"; - } - } - - /** - * @param regionName Name of a region. - * @return a pair of HRegionInfo and ServerName if regionName is - * a verified region name (we call {@link - * MetaTableAccessor#getRegionLocation(Connection, byte[])} - * else null. - * Throw IllegalArgumentException if regionName is null. - * @throws IOException - */ - Pair getRegion(final byte[] regionName) throws IOException { - if (regionName == null) { - throw new IllegalArgumentException("Pass a table name or region name"); - } - Pair pair = MetaTableAccessor.getRegion(connection, regionName); - if (pair == null) { - final AtomicReference> result = new AtomicReference<>(null); - final String encodedName = Bytes.toString(regionName); - MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { - @Override - public boolean visit(Result data) throws IOException { - RegionInfo info = MetaTableAccessor.getRegionInfo(data); - if (info == null) { - LOG.warn("No serialized HRegionInfo in " + data); - return true; - } - RegionLocations rl = MetaTableAccessor.getRegionLocations(data); - boolean matched = false; - ServerName sn = null; - if (rl != null) { - for (HRegionLocation h : rl.getRegionLocations()) { - if (h != null && encodedName.equals(h.getRegion().getEncodedName())) { - sn = h.getServerName(); - info = h.getRegion(); - matched = true; - } - } - } - if (!matched) return true; - result.set(new Pair<>(info, sn)); - return false; // found the region, stop - } - }; - - MetaTableAccessor.fullScanRegions(connection, visitor); - pair = result.get(); - } - return pair; - } - - /** - * If the input is a region name, it is returned as is. If it's an - * encoded region name, the corresponding region is found from meta - * and its region name is returned. If we can't find any region in - * meta matching the input as either region name or encoded region - * name, the input is returned as is. We don't throw unknown - * region exception. - */ - private byte[] getRegionName(final byte[] regionNameOrEncodedRegionName) throws IOException { - if (Bytes.equals(regionNameOrEncodedRegionName, - RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) || - Bytes.equals(regionNameOrEncodedRegionName, - RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) { - return RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(); - } - byte[] tmp = regionNameOrEncodedRegionName; - Pair regionServerPair = getRegion(regionNameOrEncodedRegionName); - if (regionServerPair != null && regionServerPair.getFirst() != null) { - tmp = regionServerPair.getFirst().getRegionName(); - } - return tmp; - } - - /** - * Check if table exists or not - * @param tableName Name of a table. - * @return tableName instance - * @throws IOException if a remote or network exception occurs. - * @throws TableNotFoundException if table does not exist. - */ - private TableName checkTableExists(final TableName tableName) - throws IOException { - return executeCallable(new RpcRetryingCallable() { - @Override - protected TableName rpcCall(int callTimeout) throws Exception { - if (!MetaTableAccessor.tableExists(connection, tableName)) { - throw new TableNotFoundException(tableName); - } - return tableName; - } - }); - } - - @Override - public synchronized void shutdown() throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - setPriority(HConstants.HIGH_QOS); - master.shutdown(getRpcController(), ShutdownRequest.newBuilder().build()); - return null; - } - }); - } - - @Override - public synchronized void stopMaster() throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - setPriority(HConstants.HIGH_QOS); - master.stopMaster(getRpcController(), StopMasterRequest.newBuilder().build()); - return null; - } - }); - } - - @Override - public synchronized void stopRegionServer(final String hostnamePort) - throws IOException { - String hostname = Addressing.parseHostname(hostnamePort); - int port = Addressing.parsePort(hostnamePort); - final AdminService.BlockingInterface admin = - this.connection.getAdmin(ServerName.valueOf(hostname, port, 0)); - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - controller.setPriority(HConstants.HIGH_QOS); - StopServerRequest request = RequestConverter.buildStopServerRequest( - "Called by admin client " + this.connection.toString()); - try { - admin.stopServer(controller, request); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - @Override - public boolean isMasterInMaintenanceMode() throws IOException { - return executeCallable(new MasterCallable(getConnection(), - this.rpcControllerFactory) { - @Override - protected IsInMaintenanceModeResponse rpcCall() throws Exception { - return master.isMasterInMaintenanceMode(getRpcController(), - IsInMaintenanceModeRequest.newBuilder().build()); - } - }).getInMaintenanceMode(); - } - - @Override - public ClusterMetrics getClusterMetrics(EnumSet

- * TODO: do not expose ZKConnectionException. - * @param conf system configuration - * @throws MasterNotRunningException if the master is not running. - * @throws ZooKeeperConnectionException if unable to connect to zookeeper. - */ - public static void available(final Configuration conf) - throws MasterNotRunningException, ZooKeeperConnectionException, IOException { - Configuration copyOfConf = HBaseConfiguration.create(conf); - // We set it to make it fail as soon as possible if HBase is not available - copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - copyOfConf.setInt("zookeeper.recovery.retry", 0); - - // Check ZK first. - // If the connection exists, we may have a connection to ZK that does not work anymore - try (ConnectionImplementation connection = ConnectionFactory.createConnectionImpl(copyOfConf, - null, UserProvider.instantiate(copyOfConf).getCurrent())) { - // can throw MasterNotRunningException - connection.isMasterRunning(); - } - } - - private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException, - FailedLogCloseException { - final AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest(); - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - try { - return admin.rollWALWriter(controller, request); - } catch (ServiceException e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - @Override - public synchronized void rollWALWriter(ServerName serverName) - throws IOException, FailedLogCloseException { - rollWALWriterImpl(serverName); - } - - @Override - public CompactionState getCompactionState(final TableName tableName) - throws IOException { - return getCompactionState(tableName, CompactType.NORMAL); - } - - @Override - public CompactionState getCompactionStateForRegion(final byte[] regionName) - throws IOException { - final Pair regionServerPair = getRegion(regionName); - if (regionServerPair == null) { - throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName)); - } - if (regionServerPair.getSecond() == null) { - throw new NoServerForRegionException(Bytes.toStringBinary(regionName)); - } - ServerName sn = regionServerPair.getSecond(); - final AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest( - regionServerPair.getFirst().getRegionName(), true); - GetRegionInfoResponse response; - try { - response = admin.getRegionInfo(controller, request); - } catch (ServiceException e) { - throw ProtobufUtil.handleRemoteException(e); - } - if (response.getCompactionState() != null) { - return ProtobufUtil.createCompactionState(response.getCompactionState()); - } - return null; - } - - @Override - public void snapshot(SnapshotDescription snapshotDesc) - throws IOException, SnapshotCreationException, IllegalArgumentException { - // actually take the snapshot - SnapshotProtos.SnapshotDescription snapshot = - ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc); - SnapshotResponse response = asyncSnapshot(snapshot); - final IsSnapshotDoneRequest request = - IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build(); - IsSnapshotDoneResponse done = null; - long start = EnvironmentEdgeManager.currentTime(); - long max = response.getExpectedTimeout(); - long maxPauseTime = max / this.numRetries; - int tries = 0; - LOG.debug("Waiting a max of " + max + " ms for snapshot '" + - ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " + - maxPauseTime + " ms per retry)"); - while (tries == 0 - || ((EnvironmentEdgeManager.currentTime() - start) < max && !done.getDone())) { - try { - // sleep a backoff <= pauseTime amount - long sleep = getPauseTime(tries++); - sleep = sleep > maxPauseTime ? maxPauseTime : sleep; - LOG.debug("(#" + tries + ") Sleeping: " + sleep + - "ms while waiting for snapshot completion."); - Thread.sleep(sleep); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e); - } - LOG.debug("Getting current status of snapshot from master..."); - done = executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected IsSnapshotDoneResponse rpcCall() throws Exception { - return master.isSnapshotDone(getRpcController(), request); - } - }); - } - if (!done.getDone()) { - throw new SnapshotCreationException("Snapshot '" + snapshot.getName() - + "' wasn't completed in expectedTime:" + max + " ms", snapshotDesc); - } - } - - @Override - public Future snapshotAsync(SnapshotDescription snapshotDesc) - throws IOException, SnapshotCreationException { - asyncSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc)); - return new ProcedureFuture(this, null) { - - @Override - protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException { - waitForState(deadlineTs, new WaitForStateCallable() { - - @Override - public void throwInterruptedException() throws InterruptedIOException { - throw new InterruptedIOException( - "Interrupted while waiting for taking snapshot" + snapshotDesc); - } - - @Override - public void throwTimeoutException(long elapsedTime) throws TimeoutException { - throw new TimeoutException("Snapshot '" + snapshotDesc.getName() + - "' wasn't completed in expectedTime:" + elapsedTime + " ms"); - } - - @Override - public boolean checkState(int tries) throws IOException { - return isSnapshotFinished(snapshotDesc); - } - }); - return null; - } - }; - } - - private SnapshotResponse asyncSnapshot(SnapshotProtos.SnapshotDescription snapshot) - throws IOException { - ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot); - final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot) - .build(); - // run the snapshot on the master - return executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected SnapshotResponse rpcCall() throws Exception { - return master.snapshot(getRpcController(), request); - } - }); - } - - @Override - public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc) - throws IOException, HBaseSnapshotException, UnknownSnapshotException { - final SnapshotProtos.SnapshotDescription snapshot = - ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc); - return executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected IsSnapshotDoneResponse rpcCall() throws Exception { - return master.isSnapshotDone(getRpcController(), - IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build()); - } - }).getDone(); - } - - @Override - public void restoreSnapshot(final String snapshotName) - throws IOException, RestoreSnapshotException { - boolean takeFailSafeSnapshot = - conf.getBoolean(HConstants.SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT, - HConstants.DEFAULT_SNAPSHOT_RESTORE_TAKE_FAILSAFE_SNAPSHOT); - restoreSnapshot(snapshotName, takeFailSafeSnapshot); - } - - /** - * Check whether the snapshot exists and contains disabled table - * - * @param snapshotName name of the snapshot to restore - * @throws IOException if a remote or network exception occurs - * @throws RestoreSnapshotException if no valid snapshot is found - */ - private TableName getTableNameBeforeRestoreSnapshot(final String snapshotName) - throws IOException, RestoreSnapshotException { - TableName tableName = null; - for (SnapshotDescription snapshotInfo: listSnapshots()) { - if (snapshotInfo.getName().equals(snapshotName)) { - tableName = snapshotInfo.getTableName(); - break; - } - } - - if (tableName == null) { - throw new RestoreSnapshotException( - "Unable to find the table name for snapshot=" + snapshotName); - } - return tableName; - } - - @Override - public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) - throws IOException, RestoreSnapshotException { - restoreSnapshot(snapshotName, takeFailSafeSnapshot, false); - } - - @Override - public void restoreSnapshot(final String snapshotName, final boolean takeFailSafeSnapshot, - final boolean restoreAcl) throws IOException, RestoreSnapshotException { - TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName); - - // The table does not exists, switch to clone. - if (!tableExists(tableName)) { - cloneSnapshot(snapshotName, tableName, restoreAcl); - return; - } - - // Check if the table is disabled - if (!isTableDisabled(tableName)) { - throw new TableNotDisabledException(tableName); - } - - // Take a snapshot of the current state - String failSafeSnapshotSnapshotName = null; - if (takeFailSafeSnapshot) { - failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name", - "hbase-failsafe-{snapshot.name}-{restore.timestamp}"); - failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName - .replace("{snapshot.name}", snapshotName) - .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.')) - .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime())); - LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName); - snapshot(failSafeSnapshotSnapshotName, tableName); - } - - try { - // Restore snapshot - get( - internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl), - syncWaitTimeout, - TimeUnit.MILLISECONDS); - } catch (IOException e) { - // Something went wrong during the restore... - // if the pre-restore snapshot is available try to rollback - if (takeFailSafeSnapshot) { - try { - get( - internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName, restoreAcl), - syncWaitTimeout, - TimeUnit.MILLISECONDS); - String msg = "Restore snapshot=" + snapshotName + - " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded."; - LOG.error(msg, e); - throw new RestoreSnapshotException(msg, e); - } catch (IOException ex) { - String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName; - LOG.error(msg, ex); - throw new RestoreSnapshotException(msg, e); - } - } else { - throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e); - } - } - - // If the restore is succeeded, delete the pre-restore snapshot - if (takeFailSafeSnapshot) { - try { - LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName); - deleteSnapshot(failSafeSnapshotSnapshotName); - } catch (IOException e) { - LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e); - } - } - } - - @Override - public Future cloneSnapshotAsync(String snapshotName, TableName tableName, - boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException { - if (tableExists(tableName)) { - throw new TableExistsException(tableName); - } - return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl); - } - - @Override - public byte[] execProcedureWithReturn(String signature, String instance, Map props) throws IOException { - ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props); - final ExecProcedureRequest request = - ExecProcedureRequest.newBuilder().setProcedure(desc).build(); - // run the procedure on the master - ExecProcedureResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected ExecProcedureResponse rpcCall() throws Exception { - return master.execProcedureWithRet(getRpcController(), request); - } - }); - - return response.hasReturnData() ? response.getReturnData().toByteArray() : null; - } - - @Override - public void execProcedure(String signature, String instance, Map props) - throws IOException { - ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props); - final ExecProcedureRequest request = - ExecProcedureRequest.newBuilder().setProcedure(desc).build(); - // run the procedure on the master - ExecProcedureResponse response = executeCallable(new MasterCallable( - getConnection(), getRpcControllerFactory()) { - @Override - protected ExecProcedureResponse rpcCall() throws Exception { - return master.execProcedure(getRpcController(), request); - } - }); - - long start = EnvironmentEdgeManager.currentTime(); - long max = response.getExpectedTimeout(); - long maxPauseTime = max / this.numRetries; - int tries = 0; - LOG.debug("Waiting a max of " + max + " ms for procedure '" + - signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)"); - boolean done = false; - while (tries == 0 - || ((EnvironmentEdgeManager.currentTime() - start) < max && !done)) { - try { - // sleep a backoff <= pauseTime amount - long sleep = getPauseTime(tries++); - sleep = sleep > maxPauseTime ? maxPauseTime : sleep; - LOG.debug("(#" + tries + ") Sleeping: " + sleep + - "ms while waiting for procedure completion."); - Thread.sleep(sleep); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e); - } - LOG.debug("Getting current status of procedure from master..."); - done = isProcedureFinished(signature, instance, props); - } - if (!done) { - throw new IOException("Procedure '" + signature + " : " + instance - + "' wasn't completed in expectedTime:" + max + " ms"); - } - } - - @Override - public boolean isProcedureFinished(String signature, String instance, Map props) - throws IOException { - ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props); - return executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected IsProcedureDoneResponse rpcCall() throws Exception { - return master.isProcedureDone(getRpcController(), - IsProcedureDoneRequest.newBuilder().setProcedure(desc).build()); - } - }).getDone(); - } - - /** - * Execute Restore/Clone snapshot and wait for the server to complete (blocking). - * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to - * create an HTable instance to this table before it is available. - * @param snapshotName snapshot to restore - * @param tableName table name to restore the snapshot on - * @throws IOException if a remote or network exception occurs - * @throws RestoreSnapshotException if snapshot failed to be restored - * @throws IllegalArgumentException if the restore request is formatted incorrectly - */ - private Future internalRestoreSnapshotAsync(final String snapshotName, - final TableName tableName, final boolean restoreAcl) - throws IOException, RestoreSnapshotException { - final SnapshotProtos.SnapshotDescription snapshot = - SnapshotProtos.SnapshotDescription.newBuilder() - .setName(snapshotName).setTable(tableName.getNameAsString()).build(); - - // actually restore the snapshot - ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot); - - RestoreSnapshotResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - Long nonceGroup = ng.getNonceGroup(); - Long nonce = ng.newNonce(); - @Override - protected RestoreSnapshotResponse rpcCall() throws Exception { - final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder() - .setSnapshot(snapshot) - .setNonceGroup(nonceGroup) - .setNonce(nonce) - .setRestoreACL(restoreAcl) - .build(); - return master.restoreSnapshot(getRpcController(), request); - } - }); - - return new RestoreSnapshotFuture(this, snapshot, tableName, response); - } - - private static class RestoreSnapshotFuture extends TableFuture { - public RestoreSnapshotFuture( - final HBaseAdmin admin, - final SnapshotProtos.SnapshotDescription snapshot, - final TableName tableName, - final RestoreSnapshotResponse response) { - super(admin, tableName, - (response != null && response.hasProcId()) ? response.getProcId() : null); - - if (response != null && !response.hasProcId()) { - throw new UnsupportedOperationException("Client could not call old version of Server"); - } - } - - public RestoreSnapshotFuture( - final HBaseAdmin admin, - final TableName tableName, - final Long procId) { - super(admin, tableName, procId); - } - - @Override - public String getOperationType() { - return "MODIFY"; - } - } - - @Override - public List listSnapshots() throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - List snapshotsList = master - .getCompletedSnapshots(getRpcController(), - GetCompletedSnapshotsRequest.newBuilder().build()) - .getSnapshotsList(); - List result = new ArrayList<>(snapshotsList.size()); - for (SnapshotProtos.SnapshotDescription snapshot : snapshotsList) { - result.add(ProtobufUtil.createSnapshotDesc(snapshot)); - } - return result; - } - }); - } - - @Override - public List listSnapshots(Pattern pattern) throws IOException { - List matched = new LinkedList<>(); - List snapshots = listSnapshots(); - for (SnapshotDescription snapshot : snapshots) { - if (pattern.matcher(snapshot.getName()).matches()) { - matched.add(snapshot); - } - } - return matched; - } - - @Override - public List listTableSnapshots(Pattern tableNamePattern, - Pattern snapshotNamePattern) throws IOException { - TableName[] tableNames = listTableNames(tableNamePattern); - - List tableSnapshots = new LinkedList<>(); - List snapshots = listSnapshots(snapshotNamePattern); - - List listOfTableNames = Arrays.asList(tableNames); - for (SnapshotDescription snapshot : snapshots) { - if (listOfTableNames.contains(snapshot.getTableName())) { - tableSnapshots.add(snapshot); - } - } - return tableSnapshots; - } - - @Override - public void deleteSnapshot(final String snapshotName) throws IOException { - // make sure the snapshot is possibly valid - TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName)); - // do the delete - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - master.deleteSnapshot(getRpcController(), - DeleteSnapshotRequest.newBuilder().setSnapshot( - SnapshotProtos.SnapshotDescription.newBuilder().setName(snapshotName).build()) - .build() - ); - return null; - } - }); - } - - @Override - public void deleteSnapshots(final Pattern pattern) throws IOException { - List snapshots = listSnapshots(pattern); - for (final SnapshotDescription snapshot : snapshots) { - try { - internalDeleteSnapshot(snapshot); - } catch (IOException ex) { - LOG.info("Failed to delete snapshot " + snapshot.getName() + " for table " - + snapshot.getTableNameAsString(), ex); - } - } - } - - private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - this.master.deleteSnapshot(getRpcController(), DeleteSnapshotRequest.newBuilder() - .setSnapshot(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot)).build()); - return null; - } - }); - } - - @Override - public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) - throws IOException { - List snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern); - for (SnapshotDescription snapshot : snapshots) { - try { - internalDeleteSnapshot(snapshot); - LOG.debug("Successfully deleted snapshot: " + snapshot.getName()); - } catch (IOException e) { - LOG.error("Failed to delete snapshot: " + snapshot.getName(), e); - } - } - } - - @Override - public void setQuota(final QuotaSettings quota) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - this.master.setQuota(getRpcController(), QuotaSettings.buildSetQuotaRequestProto(quota)); - return null; - } - }); - } - - @Override - public List getQuota(QuotaFilter filter) throws IOException { - List quotas = new LinkedList<>(); - try (QuotaRetriever retriever = QuotaRetriever.open(conf, filter)) { - Iterator iterator = retriever.iterator(); - while (iterator.hasNext()) { - quotas.add(iterator.next()); - } - } - return quotas; - } - - private & Closeable, V> V executeCallable(C callable) - throws IOException { - return executeCallable(callable, rpcCallerFactory, operationTimeout, rpcTimeout); - } - - static private & Closeable, V> V executeCallable(C callable, - RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout) - throws IOException { - RpcRetryingCaller caller = rpcCallerFactory.newCaller(rpcTimeout); - try { - return caller.callWithRetries(callable, operationTimeout); - } finally { - callable.close(); - } - } - - @Override - // Coprocessor Endpoint against the Master. - public CoprocessorRpcChannel coprocessorService() { - return new SyncCoprocessorRpcChannel() { - @Override - protected Message callExecService(final RpcController controller, - final Descriptors.MethodDescriptor method, final Message request, - final Message responsePrototype) - throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("Call: " + method.getName() + ", " + request.toString()); - } - // Try-with-resources so close gets called when we are done. - try (MasterCallable callable = - new MasterCallable(connection, - connection.getRpcControllerFactory()) { - @Override - protected CoprocessorServiceResponse rpcCall() throws Exception { - CoprocessorServiceRequest csr = - CoprocessorRpcUtils.getCoprocessorServiceRequest(method, request); - return this.master.execMasterService(getRpcController(), csr); - } - }) { - // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller - callable.prepare(false); - int operationTimeout = connection.getConnectionConfiguration().getOperationTimeout(); - CoprocessorServiceResponse result = callable.call(operationTimeout); - return CoprocessorRpcUtils.getResponse(result, responsePrototype); - } - } - }; - } - - @Override - public CoprocessorRpcChannel coprocessorService(final ServerName serverName) { - return new SyncCoprocessorRpcChannel() { - @Override - protected Message callExecService(RpcController controller, - Descriptors.MethodDescriptor method, Message request, Message responsePrototype) - throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("Call: " + method.getName() + ", " + request.toString()); - } - CoprocessorServiceRequest csr = - CoprocessorRpcUtils.getCoprocessorServiceRequest(method, request); - // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller - // TODO: Make this same as RegionCoprocessorRpcChannel and MasterCoprocessorRpcChannel. They - // are all different though should do same thing; e.g. RpcChannel setup. - ClientProtos.ClientService.BlockingInterface stub = connection.getClient(serverName); - CoprocessorServiceResponse result; - try { - result = stub. - execRegionServerService(connection.getRpcControllerFactory().newController(), csr); - return CoprocessorRpcUtils.getResponse(result, responsePrototype); - } catch (ServiceException e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - }; - } - - @Override - public void updateConfiguration(final ServerName server) throws IOException { - final AdminService.BlockingInterface admin = this.connection.getAdmin(server); - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - admin.updateConfiguration(null, UpdateConfigurationRequest.getDefaultInstance()); - return null; - } - }; - ProtobufUtil.call(callable); - } - - @Override - public void updateConfiguration() throws IOException { - ClusterMetrics status = getClusterMetrics( - EnumSet.of(Option.LIVE_SERVERS, Option.MASTER, Option.BACKUP_MASTERS)); - for (ServerName server : status.getLiveServerMetrics().keySet()) { - updateConfiguration(server); - } - - updateConfiguration(status.getMasterName()); - - for (ServerName server : status.getBackupMasterNames()) { - updateConfiguration(server); - } - } - - @Override - public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Long rpcCall() throws Exception { - MajorCompactionTimestampRequest req = - MajorCompactionTimestampRequest.newBuilder() - .setTableName(ProtobufUtil.toProtoTableName(tableName)).build(); - return master.getLastMajorCompactionTimestamp(getRpcController(), req). - getCompactionTimestamp(); - } - }); - } - - @Override - public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Long rpcCall() throws Exception { - MajorCompactionTimestampForRegionRequest req = - MajorCompactionTimestampForRegionRequest.newBuilder().setRegion(RequestConverter - .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build(); - return master.getLastMajorCompactionTimestampForRegion(getRpcController(), req) - .getCompactionTimestamp(); - } - }); - } - - /** - * {@inheritDoc} - */ - @Override - public void compact(final TableName tableName, final byte[] columnFamily, CompactType compactType) - throws IOException, InterruptedException { - compact(tableName, columnFamily, false, compactType); - } - - /** - * {@inheritDoc} - */ - @Override - public void compact(final TableName tableName, CompactType compactType) - throws IOException, InterruptedException { - compact(tableName, null, false, compactType); - } - - /** - * {@inheritDoc} - */ - @Override - public void majorCompact(final TableName tableName, final byte[] columnFamily, - CompactType compactType) throws IOException, InterruptedException { - compact(tableName, columnFamily, true, compactType); - } - - /** - * {@inheritDoc} - */ - @Override - public void majorCompact(final TableName tableName, CompactType compactType) - throws IOException, InterruptedException { - compact(tableName, null, true, compactType); - } - - /** - * {@inheritDoc} - */ - @Override - public CompactionState getCompactionState(final TableName tableName, CompactType compactType) - throws IOException { - checkTableExists(tableName); - if (!isTableEnabled(tableName)) { - // If the table is disabled, the compaction state of the table should always be NONE - return ProtobufUtil.createCompactionState( - AdminProtos.GetRegionInfoResponse.CompactionState.NONE); - } - - AdminProtos.GetRegionInfoResponse.CompactionState state = - AdminProtos.GetRegionInfoResponse.CompactionState.NONE; - - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController rpcController = rpcControllerFactory.newController(); - switch (compactType) { - case MOB: - final AdminProtos.AdminService.BlockingInterface masterAdmin = - this.connection.getAdminForMaster(); - Callable callable = - new Callable() { - @Override - public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception { - RegionInfo info = RegionInfo.createMobRegionInfo(tableName); - GetRegionInfoRequest request = - RequestConverter.buildGetRegionInfoRequest(info.getRegionName(), true); - GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request); - return response.getCompactionState(); - } - }; - state = ProtobufUtil.call(callable); - break; - case NORMAL: - for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) { - ServerName sn = loc.getServerName(); - if (sn == null) { - continue; - } - byte[] regionName = loc.getRegion().getRegionName(); - AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn); - try { - Callable regionInfoCallable = - new Callable() { - @Override - public GetRegionInfoResponse call() throws Exception { - GetRegionInfoRequest request = - RequestConverter.buildGetRegionInfoRequest(regionName, true); - return snAdmin.getRegionInfo(rpcController, request); - } - }; - GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable); - switch (response.getCompactionState()) { - case MAJOR_AND_MINOR: - return CompactionState.MAJOR_AND_MINOR; - case MAJOR: - if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MINOR) { - return CompactionState.MAJOR_AND_MINOR; - } - state = AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR; - break; - case MINOR: - if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR) { - return CompactionState.MAJOR_AND_MINOR; - } - state = AdminProtos.GetRegionInfoResponse.CompactionState.MINOR; - break; - case NONE: - default: // nothing, continue - } - } catch (NotServingRegionException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to get compaction state of " + loc.getRegion() + ": " + - StringUtils.stringifyException(e)); - } - } catch (RemoteException e) { - if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to get compaction state of " + loc.getRegion() + ": " + - StringUtils.stringifyException(e)); - } - } else { - throw e; - } - } - } - break; - default: - throw new IllegalArgumentException("Unknown compactType: " + compactType); - } - if (state != null) { - return ProtobufUtil.createCompactionState(state); - } - return null; - } - - /** - * Future that waits on a procedure result. - * Returned by the async version of the Admin calls, - * and used internally by the sync calls to wait on the result of the procedure. - */ - @InterfaceAudience.Private - @InterfaceStability.Evolving - protected static class ProcedureFuture implements Future { - private ExecutionException exception = null; - private boolean procResultFound = false; - private boolean done = false; - private boolean cancelled = false; - private V result = null; - - private final HBaseAdmin admin; - protected final Long procId; - - public ProcedureFuture(final HBaseAdmin admin, final Long procId) { - this.admin = admin; - this.procId = procId; - } - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - AbortProcedureRequest abortProcRequest = AbortProcedureRequest.newBuilder() - .setProcId(procId).setMayInterruptIfRunning(mayInterruptIfRunning).build(); - try { - cancelled = abortProcedureResult(abortProcRequest).getIsProcedureAborted(); - if (cancelled) { - done = true; - } - } catch (IOException e) { - // Cancell thrown exception for some reason. At this time, we are not sure whether - // the cancell succeeds or fails. We assume that it is failed, but print out a warning - // for debugging purpose. - LOG.warn( - "Cancelling the procedure with procId=" + procId + " throws exception " + e.getMessage(), - e); - cancelled = false; - } - return cancelled; - } - - @Override - public boolean isCancelled() { - return cancelled; - } - - protected AbortProcedureResponse abortProcedureResult( - final AbortProcedureRequest request) throws IOException { - return admin.executeCallable(new MasterCallable( - admin.getConnection(), admin.getRpcControllerFactory()) { - @Override - protected AbortProcedureResponse rpcCall() throws Exception { - return master.abortProcedure(getRpcController(), request); - } - }); - } - - @Override - public V get() throws InterruptedException, ExecutionException { - // TODO: should we ever spin forever? - // fix HBASE-21715. TODO: If the function call get() without timeout limit is not allowed, - // is it possible to compose instead of inheriting from the class Future for this class? - try { - return get(admin.getProcedureTimeout, TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - LOG.warn("Failed to get the procedure with procId=" + procId + " throws exception " + e - .getMessage(), e); - return null; - } - } - - @Override - public V get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - if (!done) { - long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout); - try { - try { - // if the master support procedures, try to wait the result - if (procId != null) { - result = waitProcedureResult(procId, deadlineTs); - } - // if we don't have a proc result, try the compatibility wait - if (!procResultFound) { - result = waitOperationResult(deadlineTs); - } - result = postOperationResult(result, deadlineTs); - done = true; - } catch (IOException e) { - result = postOperationFailure(e, deadlineTs); - done = true; - } - } catch (IOException e) { - exception = new ExecutionException(e); - done = true; - } - } - if (exception != null) { - throw exception; - } - return result; - } - - @Override - public boolean isDone() { - return done; - } - - protected HBaseAdmin getAdmin() { - return admin; - } - - private V waitProcedureResult(long procId, long deadlineTs) - throws IOException, TimeoutException, InterruptedException { - GetProcedureResultRequest request = GetProcedureResultRequest.newBuilder() - .setProcId(procId) - .build(); - - int tries = 0; - IOException serviceEx = null; - while (EnvironmentEdgeManager.currentTime() < deadlineTs) { - GetProcedureResultResponse response = null; - try { - // Try to fetch the result - response = getProcedureResult(request); - } catch (IOException e) { - serviceEx = unwrapException(e); - - // the master may be down - LOG.warn("failed to get the procedure result procId=" + procId, serviceEx); - - // Not much to do, if we have a DoNotRetryIOException - if (serviceEx instanceof DoNotRetryIOException) { - // TODO: looks like there is no way to unwrap this exception and get the proper - // UnsupportedOperationException aside from looking at the message. - // anyway, if we fail here we just failover to the compatibility side - // and that is always a valid solution. - LOG.warn("Proc-v2 is unsupported on this master: " + serviceEx.getMessage(), serviceEx); - procResultFound = false; - return null; - } - } - - // If the procedure is no longer running, we should have a result - if (response != null && response.getState() != GetProcedureResultResponse.State.RUNNING) { - procResultFound = response.getState() != GetProcedureResultResponse.State.NOT_FOUND; - return convertResult(response); - } - - try { - Thread.sleep(getAdmin().getPauseTime(tries++)); - } catch (InterruptedException e) { - throw new InterruptedException( - "Interrupted while waiting for the result of proc " + procId); - } - } - if (serviceEx != null) { - throw serviceEx; - } else { - throw new TimeoutException("The procedure " + procId + " is still running"); - } - } - - private static IOException unwrapException(IOException e) { - if (e instanceof RemoteException) { - return ((RemoteException)e).unwrapRemoteException(); - } - return e; - } - - protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request) - throws IOException { - return admin.executeCallable(new MasterCallable( - admin.getConnection(), admin.getRpcControllerFactory()) { - @Override - protected GetProcedureResultResponse rpcCall() throws Exception { - return master.getProcedureResult(getRpcController(), request); - } - }); - } - - /** - * Convert the procedure result response to a specified type. - * @param response the procedure result object to parse - * @return the result data of the procedure. - */ - protected V convertResult(final GetProcedureResultResponse response) throws IOException { - if (response.hasException()) { - throw ForeignExceptionUtil.toIOException(response.getException()); - } - return null; - } - - /** - * Fallback implementation in case the procedure is not supported by the server. - * It should try to wait until the operation is completed. - * @param deadlineTs the timestamp after which this method should throw a TimeoutException - * @return the result data of the operation - */ - protected V waitOperationResult(final long deadlineTs) - throws IOException, TimeoutException { - return null; - } - - /** - * Called after the operation is completed and the result fetched. this allows to perform extra - * steps after the procedure is completed. it allows to apply transformations to the result that - * will be returned by get(). - * @param result the result of the procedure - * @param deadlineTs the timestamp after which this method should throw a TimeoutException - * @return the result of the procedure, which may be the same as the passed one - */ - protected V postOperationResult(final V result, final long deadlineTs) - throws IOException, TimeoutException { - return result; - } - - /** - * Called after the operation is terminated with a failure. - * this allows to perform extra steps after the procedure is terminated. - * it allows to apply transformations to the result that will be returned by get(). - * The default implementation will rethrow the exception - * @param exception the exception got from fetching the result - * @param deadlineTs the timestamp after which this method should throw a TimeoutException - * @return the result of the procedure, which may be the same as the passed one - */ - protected V postOperationFailure(final IOException exception, final long deadlineTs) - throws IOException, TimeoutException { - throw exception; - } - - protected interface WaitForStateCallable { - boolean checkState(int tries) throws IOException; - void throwInterruptedException() throws InterruptedIOException; - void throwTimeoutException(long elapsed) throws TimeoutException; - } - - protected void waitForState(final long deadlineTs, final WaitForStateCallable callable) - throws IOException, TimeoutException { - int tries = 0; - IOException serverEx = null; - long startTime = EnvironmentEdgeManager.currentTime(); - while (EnvironmentEdgeManager.currentTime() < deadlineTs) { - serverEx = null; - try { - if (callable.checkState(tries)) { - return; - } - } catch (IOException e) { - serverEx = e; - } - try { - Thread.sleep(getAdmin().getPauseTime(tries++)); - } catch (InterruptedException e) { - callable.throwInterruptedException(); - } - } - if (serverEx != null) { - throw unwrapException(serverEx); - } else { - callable.throwTimeoutException(EnvironmentEdgeManager.currentTime() - startTime); - } - } - } - - @InterfaceAudience.Private - @InterfaceStability.Evolving - protected static abstract class TableFuture extends ProcedureFuture { - private final TableName tableName; - - public TableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) { - super(admin, procId); - this.tableName = tableName; - } - - @Override - public String toString() { - return getDescription(); - } - - /** - * @return the table name - */ - protected TableName getTableName() { - return tableName; - } - - /** - * @return the table descriptor - */ - protected TableDescriptor getDescriptor() throws IOException { - return getAdmin().getDescriptor(getTableName()); - } - - /** - * @return the operation type like CREATE, DELETE, DISABLE etc. - */ - public abstract String getOperationType(); - - /** - * @return a description of the operation - */ - protected String getDescription() { - return "Operation: " + getOperationType() + ", " + "Table Name: " + - tableName.getNameWithNamespaceInclAsString() + ", procId: " + procId; - } - - protected abstract class TableWaitForStateCallable implements WaitForStateCallable { - @Override - public void throwInterruptedException() throws InterruptedIOException { - throw new InterruptedIOException("Interrupted while waiting for " + getDescription()); - } - - @Override - public void throwTimeoutException(long elapsedTime) throws TimeoutException { - throw new TimeoutException( - getDescription() + " has not completed after " + elapsedTime + "ms"); - } - } - - @Override - protected V postOperationResult(final V result, final long deadlineTs) - throws IOException, TimeoutException { - LOG.info(getDescription() + " completed"); - return super.postOperationResult(result, deadlineTs); - } - - @Override - protected V postOperationFailure(final IOException exception, final long deadlineTs) - throws IOException, TimeoutException { - LOG.info(getDescription() + " failed with " + exception.getMessage()); - return super.postOperationFailure(exception, deadlineTs); - } - - protected void waitForTableEnabled(final long deadlineTs) - throws IOException, TimeoutException { - waitForState(deadlineTs, new TableWaitForStateCallable() { - @Override - public boolean checkState(int tries) throws IOException { - try { - if (getAdmin().isTableAvailable(tableName)) { - return true; - } - } catch (TableNotFoundException tnfe) { - LOG.debug("Table " + tableName.getNameWithNamespaceInclAsString() - + " was not enabled, sleeping. tries=" + tries); - } - return false; - } - }); - } - - protected void waitForTableDisabled(final long deadlineTs) - throws IOException, TimeoutException { - waitForState(deadlineTs, new TableWaitForStateCallable() { - @Override - public boolean checkState(int tries) throws IOException { - return getAdmin().isTableDisabled(tableName); - } - }); - } - - protected void waitTableNotFound(final long deadlineTs) - throws IOException, TimeoutException { - waitForState(deadlineTs, new TableWaitForStateCallable() { - @Override - public boolean checkState(int tries) throws IOException { - return !getAdmin().tableExists(tableName); - } - }); - } - - protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys) - throws IOException, TimeoutException { - final TableDescriptor desc = getDescriptor(); - final AtomicInteger actualRegCount = new AtomicInteger(0); - final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { - @Override - public boolean visit(Result rowResult) throws IOException { - RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult); - if (list == null) { - LOG.warn("No serialized HRegionInfo in " + rowResult); - return true; - } - HRegionLocation l = list.getRegionLocation(); - if (l == null) { - return true; - } - if (!l.getRegion().getTable().equals(desc.getTableName())) { - return false; - } - if (l.getRegion().isOffline() || l.getRegion().isSplit()) { - return true; - } - HRegionLocation[] locations = list.getRegionLocations(); - for (HRegionLocation location : locations) { - if (location == null) continue; - ServerName serverName = location.getServerName(); - // Make sure that regions are assigned to server - if (serverName != null && serverName.getAddress() != null) { - actualRegCount.incrementAndGet(); - } - } - return true; - } - }; - - int tries = 0; - int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication(); - while (EnvironmentEdgeManager.currentTime() < deadlineTs) { - actualRegCount.set(0); - MetaTableAccessor.scanMetaForTableRegions(getAdmin().getConnection(), visitor, - desc.getTableName()); - if (actualRegCount.get() == numRegs) { - // all the regions are online - return; - } - - try { - Thread.sleep(getAdmin().getPauseTime(tries++)); - } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted when opening" + " regions; " - + actualRegCount.get() + " of " + numRegs + " regions processed so far"); - } - } - throw new TimeoutException("Only " + actualRegCount.get() + " of " + numRegs - + " regions are online; retries exhausted."); - } - } - - @InterfaceAudience.Private - @InterfaceStability.Evolving - protected static abstract class NamespaceFuture extends ProcedureFuture { - private final String namespaceName; - - public NamespaceFuture(final HBaseAdmin admin, final String namespaceName, final Long procId) { - super(admin, procId); - this.namespaceName = namespaceName; - } - - /** - * @return the namespace name - */ - protected String getNamespaceName() { - return namespaceName; - } - - /** - * @return the operation type like CREATE_NAMESPACE, DELETE_NAMESPACE, etc. - */ - public abstract String getOperationType(); - - @Override - public String toString() { - return "Operation: " + getOperationType() + ", Namespace: " + getNamespaceName(); - } - } - - @InterfaceAudience.Private - @InterfaceStability.Evolving - private static class ReplicationFuture extends ProcedureFuture { - private final String peerId; - private final Supplier getOperation; - - public ReplicationFuture(HBaseAdmin admin, String peerId, Long procId, - Supplier getOperation) { - super(admin, procId); - this.peerId = peerId; - this.getOperation = getOperation; - } - - @Override - public String toString() { - return "Operation: " + getOperation.get() + ", peerId: " + peerId; - } - } - - @Override - public List getSecurityCapabilities() throws IOException { - try { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - SecurityCapabilitiesRequest req = SecurityCapabilitiesRequest.newBuilder().build(); - return ProtobufUtil.toSecurityCapabilityList( - master.getSecurityCapabilities(getRpcController(), req).getCapabilitiesList()); - } - }); - } catch (IOException e) { - if (e instanceof RemoteException) { - e = ((RemoteException)e).unwrapRemoteException(); - } - throw e; - } - } - - @Override - public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException { - return splitOrMergeSwitch(enabled, synchronous, MasterSwitchType.SPLIT); - } - - @Override - public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException { - return splitOrMergeSwitch(enabled, synchronous, MasterSwitchType.MERGE); - } - - private boolean splitOrMergeSwitch(boolean enabled, boolean synchronous, - MasterSwitchType switchType) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled( - getRpcController(), - RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchType)); - return response.getPrevValueList().get(0); - } - }); - } - - @Override - public boolean isSplitEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.isSplitOrMergeEnabled(getRpcController(), - RequestConverter.buildIsSplitOrMergeEnabledRequest(MasterSwitchType.SPLIT)).getEnabled(); - } - }); - } - - @Override - public boolean isMergeEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return master.isSplitOrMergeEnabled(getRpcController(), - RequestConverter.buildIsSplitOrMergeEnabledRequest(MasterSwitchType.MERGE)).getEnabled(); - } - }); - } - - private RpcControllerFactory getRpcControllerFactory() { - return this.rpcControllerFactory; - } - - @Override - public Future addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, - boolean enabled) throws IOException { - AddReplicationPeerResponse response = executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected AddReplicationPeerResponse rpcCall() throws Exception { - return master.addReplicationPeer(getRpcController(), - RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled)); - } - }); - return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER"); - } - - @Override - public Future removeReplicationPeerAsync(String peerId) throws IOException { - RemoveReplicationPeerResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected RemoveReplicationPeerResponse rpcCall() throws Exception { - return master.removeReplicationPeer(getRpcController(), - RequestConverter.buildRemoveReplicationPeerRequest(peerId)); - } - }); - return new ReplicationFuture(this, peerId, response.getProcId(), - () -> "REMOVE_REPLICATION_PEER"); - } - - @Override - public Future enableReplicationPeerAsync(final String peerId) throws IOException { - EnableReplicationPeerResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected EnableReplicationPeerResponse rpcCall() throws Exception { - return master.enableReplicationPeer(getRpcController(), - RequestConverter.buildEnableReplicationPeerRequest(peerId)); - } - }); - return new ReplicationFuture(this, peerId, response.getProcId(), - () -> "ENABLE_REPLICATION_PEER"); - } - - @Override - public Future disableReplicationPeerAsync(final String peerId) throws IOException { - DisableReplicationPeerResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected DisableReplicationPeerResponse rpcCall() throws Exception { - return master.disableReplicationPeer(getRpcController(), - RequestConverter.buildDisableReplicationPeerRequest(peerId)); - } - }); - return new ReplicationFuture(this, peerId, response.getProcId(), - () -> "DISABLE_REPLICATION_PEER"); - } - - @Override - public ReplicationPeerConfig getReplicationPeerConfig(final String peerId) throws IOException { - return executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected ReplicationPeerConfig rpcCall() throws Exception { - GetReplicationPeerConfigResponse response = master.getReplicationPeerConfig( - getRpcController(), RequestConverter.buildGetReplicationPeerConfigRequest(peerId)); - return ReplicationPeerConfigUtil.convert(response.getPeerConfig()); - } - }); - } - - @Override - public Future updateReplicationPeerConfigAsync(final String peerId, - final ReplicationPeerConfig peerConfig) throws IOException { - UpdateReplicationPeerConfigResponse response = - executeCallable(new MasterCallable(getConnection(), - getRpcControllerFactory()) { - @Override - protected UpdateReplicationPeerConfigResponse rpcCall() throws Exception { - return master.updateReplicationPeerConfig(getRpcController(), - RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig)); - } - }); - return new ReplicationFuture(this, peerId, response.getProcId(), - () -> "UPDATE_REPLICATION_PEER_CONFIG"); - } - - @Override - public Future transitReplicationPeerSyncReplicationStateAsync(String peerId, - SyncReplicationState state) throws IOException { - TransitReplicationPeerSyncReplicationStateResponse response = - executeCallable(new MasterCallable( - getConnection(), getRpcControllerFactory()) { - @Override - protected TransitReplicationPeerSyncReplicationStateResponse rpcCall() throws Exception { - return master.transitReplicationPeerSyncReplicationState(getRpcController(), - RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId, - state)); - } - }); - return new ReplicationFuture(this, peerId, response.getProcId(), - () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"); - } - - @Override - public List listReplicationPeers() throws IOException { - return listReplicationPeers((Pattern)null); - } - - @Override - public List listReplicationPeers(Pattern pattern) - throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - List peersList = master.listReplicationPeers( - getRpcController(), RequestConverter.buildListReplicationPeersRequest(pattern)) - .getPeerDescList(); - List result = new ArrayList<>(peersList.size()); - for (ReplicationProtos.ReplicationPeerDescription peer : peersList) { - result.add(ReplicationPeerConfigUtil.toReplicationPeerDescription(peer)); - } - return result; - } - }); - } - - @Override - public void decommissionRegionServers(List servers, boolean offload) - throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - public Void rpcCall() throws ServiceException { - master.decommissionRegionServers(getRpcController(), - RequestConverter.buildDecommissionRegionServersRequest(servers, offload)); - return null; - } - }); - } - - @Override - public List listDecommissionedRegionServers() throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - public List rpcCall() throws ServiceException { - ListDecommissionedRegionServersRequest req = - ListDecommissionedRegionServersRequest.newBuilder().build(); - List servers = new ArrayList<>(); - for (HBaseProtos.ServerName server : master - .listDecommissionedRegionServers(getRpcController(), req).getServerNameList()) { - servers.add(ProtobufUtil.toServerName(server)); - } - return servers; - } - }); - } - - @Override - public void recommissionRegionServer(ServerName server, List encodedRegionNames) - throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - public Void rpcCall() throws ServiceException { - master.recommissionRegionServer(getRpcController(), - RequestConverter.buildRecommissionRegionServerRequest(server, encodedRegionNames)); - return null; - } - }); - } - - @Override - public List listReplicatedTableCFs() throws IOException { - List replicatedTableCFs = new ArrayList<>(); - List tables = listTableDescriptors(); - tables.forEach(table -> { - Map cfs = new HashMap<>(); - Stream.of(table.getColumnFamilies()) - .filter(column -> column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) - .forEach(column -> { - cfs.put(column.getNameAsString(), column.getScope()); - }); - if (!cfs.isEmpty()) { - replicatedTableCFs.add(new TableCFs(table.getTableName(), cfs)); - } - }); - return replicatedTableCFs; - } - - @Override - public void enableTableReplication(final TableName tableName) throws IOException { - if (tableName == null) { - throw new IllegalArgumentException("Table name cannot be null"); - } - if (!tableExists(tableName)) { - throw new TableNotFoundException("Table '" + tableName.getNameAsString() - + "' does not exists."); - } - byte[][] splits = getTableSplits(tableName); - checkAndSyncTableDescToPeers(tableName, splits); - setTableRep(tableName, true); - } - - @Override - public void disableTableReplication(final TableName tableName) throws IOException { - if (tableName == null) { - throw new IllegalArgumentException("Table name is null"); - } - if (!tableExists(tableName)) { - throw new TableNotFoundException("Table '" + tableName.getNameAsString() - + "' does not exists."); - } - setTableRep(tableName, false); - } - - /** - * Connect to peer and check the table descriptor on peer: - *

    - *
  1. Create the same table on peer when not exist.
  2. - *
  3. Throw an exception if the table already has replication enabled on any of the column - * families.
  4. - *
  5. Throw an exception if the table exists on peer cluster but descriptors are not same.
  6. - *
- * @param tableName name of the table to sync to the peer - * @param splits table split keys - * @throws IOException - */ - private void checkAndSyncTableDescToPeers(final TableName tableName, final byte[][] splits) - throws IOException { - List peers = listReplicationPeers(); - if (peers == null || peers.size() <= 0) { - throw new IllegalArgumentException("Found no peer cluster for replication."); - } - - for (ReplicationPeerDescription peerDesc : peers) { - if (peerDesc.getPeerConfig().needToReplicate(tableName)) { - Configuration peerConf = - ReplicationPeerConfigUtil.getPeerClusterConfiguration(this.conf, peerDesc); - try (Connection conn = ConnectionFactory.createConnection(peerConf); - Admin repHBaseAdmin = conn.getAdmin()) { - TableDescriptor tableDesc = getDescriptor(tableName); - TableDescriptor peerTableDesc = null; - if (!repHBaseAdmin.tableExists(tableName)) { - repHBaseAdmin.createTable(tableDesc, splits); - } else { - peerTableDesc = repHBaseAdmin.getDescriptor(tableName); - if (peerTableDesc == null) { - throw new IllegalArgumentException("Failed to get table descriptor for table " - + tableName.getNameAsString() + " from peer cluster " + peerDesc.getPeerId()); - } - if (TableDescriptor.COMPARATOR_IGNORE_REPLICATION.compare(peerTableDesc, - tableDesc) != 0) { - throw new IllegalArgumentException("Table " + tableName.getNameAsString() - + " exists in peer cluster " + peerDesc.getPeerId() - + ", but the table descriptors are not same when compared with source cluster." - + " Thus can not enable the table's replication switch."); - } - } - } - } - } - } - - /** - * Set the table's replication switch if the table's replication switch is already not set. - * @param tableName name of the table - * @param enableRep is replication switch enable or disable - * @throws IOException if a remote or network exception occurs - */ - private void setTableRep(final TableName tableName, boolean enableRep) throws IOException { - TableDescriptor tableDesc = getDescriptor(tableName); - if (!tableDesc.matchReplicationScope(enableRep)) { - int scope = - enableRep ? HConstants.REPLICATION_SCOPE_GLOBAL : HConstants.REPLICATION_SCOPE_LOCAL; - modifyTable(TableDescriptorBuilder.newBuilder(tableDesc).setReplicationScope(scope).build()); - } - } - - @Override - public void clearCompactionQueues(final ServerName sn, final Set queues) - throws IOException, InterruptedException { - if (queues == null || queues.size() == 0) { - throw new IllegalArgumentException("queues cannot be null or empty"); - } - final AdminService.BlockingInterface admin = this.connection.getAdmin(sn); - Callable callable = new Callable() { - @Override - public Void call() throws Exception { - // TODO: There is no timeout on this controller. Set one! - HBaseRpcController controller = rpcControllerFactory.newController(); - ClearCompactionQueuesRequest request = - RequestConverter.buildClearCompactionQueuesRequest(queues); - admin.clearCompactionQueues(controller, request); - return null; - } - }; - ProtobufUtil.call(callable); - } - - @Override - public List clearDeadServers(List servers) throws IOException { - return executeCallable(new MasterCallable>(getConnection(), - getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - ClearDeadServersRequest req = RequestConverter. - buildClearDeadServersRequest(servers == null? Collections.emptyList(): servers); - return ProtobufUtil.toServerNameList( - master.clearDeadServers(getRpcController(), req).getServerNameList()); - } - }); - } - - @Override - public void cloneTableSchema(final TableName tableName, final TableName newTableName, - final boolean preserveSplits) throws IOException { - checkTableExists(tableName); - if (tableExists(newTableName)) { - throw new TableExistsException(newTableName); - } - TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getDescriptor(tableName)); - if (preserveSplits) { - createTable(htd, getTableSplits(tableName)); - } else { - createTable(htd); - } - } - - @Override - public boolean switchRpcThrottle(final boolean enable) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return this.master - .switchRpcThrottle(getRpcController(), MasterProtos.SwitchRpcThrottleRequest - .newBuilder().setRpcThrottleEnabled(enable).build()) - .getPreviousRpcThrottleEnabled(); - } - }); - } - - @Override - public boolean isRpcThrottleEnabled() throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return this.master.isRpcThrottleEnabled(getRpcController(), - IsRpcThrottleEnabledRequest.newBuilder().build()).getRpcThrottleEnabled(); - } - }); - } - - @Override - public boolean exceedThrottleQuotaSwitch(final boolean enable) throws IOException { - return executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Boolean rpcCall() throws Exception { - return this.master - .switchExceedThrottleQuota(getRpcController(), - MasterProtos.SwitchExceedThrottleQuotaRequest.newBuilder() - .setExceedThrottleQuotaEnabled(enable).build()) - .getPreviousExceedThrottleQuotaEnabled(); - } - }); - } - - @Override - public Map getSpaceQuotaTableSizes() throws IOException { - return executeCallable( - new MasterCallable>(getConnection(), getRpcControllerFactory()) { - @Override - protected Map rpcCall() throws Exception { - GetSpaceQuotaRegionSizesResponse resp = master.getSpaceQuotaRegionSizes( - getRpcController(), RequestConverter.buildGetSpaceQuotaRegionSizesRequest()); - Map tableSizes = new HashMap<>(); - for (RegionSizes sizes : resp.getSizesList()) { - TableName tn = ProtobufUtil.toTableName(sizes.getTableName()); - tableSizes.put(tn, sizes.getSize()); - } - return tableSizes; - } - }); - } - - @Override - public Map getRegionServerSpaceQuotaSnapshots( - ServerName serverName) throws IOException { - final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName); - Callable callable = - new Callable() { - @Override - public GetSpaceQuotaSnapshotsResponse call() throws Exception { - return admin.getSpaceQuotaSnapshots(rpcControllerFactory.newController(), - RequestConverter.buildGetSpaceQuotaSnapshotsRequest()); - } - }; - GetSpaceQuotaSnapshotsResponse resp = ProtobufUtil.call(callable); - Map snapshots = new HashMap<>(); - for (TableQuotaSnapshot snapshot : resp.getSnapshotsList()) { - snapshots.put(ProtobufUtil.toTableName(snapshot.getTableName()), - SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot.getSnapshot())); - } - return snapshots; - } - - @Override - public SpaceQuotaSnapshot getCurrentSpaceQuotaSnapshot(String namespace) throws IOException { - return executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected SpaceQuotaSnapshot rpcCall() throws Exception { - GetQuotaStatesResponse resp = master.getQuotaStates(getRpcController(), - RequestConverter.buildGetQuotaStatesRequest()); - for (GetQuotaStatesResponse.NamespaceQuotaSnapshot nsSnapshot : resp - .getNsSnapshotsList()) { - if (namespace.equals(nsSnapshot.getNamespace())) { - return SpaceQuotaSnapshot.toSpaceQuotaSnapshot(nsSnapshot.getSnapshot()); - } - } - return null; - } - }); - } - - @Override - public SpaceQuotaSnapshot getCurrentSpaceQuotaSnapshot(TableName tableName) throws IOException { - return executeCallable( - new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected SpaceQuotaSnapshot rpcCall() throws Exception { - GetQuotaStatesResponse resp = master.getQuotaStates(getRpcController(), - RequestConverter.buildGetQuotaStatesRequest()); - HBaseProtos.TableName protoTableName = ProtobufUtil.toProtoTableName(tableName); - for (GetQuotaStatesResponse.TableQuotaSnapshot tableSnapshot : resp - .getTableSnapshotsList()) { - if (protoTableName.equals(tableSnapshot.getTableName())) { - return SpaceQuotaSnapshot.toSpaceQuotaSnapshot(tableSnapshot.getSnapshot()); - } - } - return null; - } - }); - } - - @Override - public void grant(UserPermission userPermission, boolean mergeExistingPermissions) - throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - GrantRequest req = - ShadedAccessControlUtil.buildGrantRequest(userPermission, mergeExistingPermissions); - this.master.grant(getRpcController(), req); - return null; - } - }); - } - - @Override - public void revoke(UserPermission userPermission) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - RevokeRequest req = ShadedAccessControlUtil.buildRevokeRequest(userPermission); - this.master.revoke(getRpcController(), req); - return null; - } - }); - } - - @Override - public List - getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException { - return executeCallable( - new MasterCallable>(getConnection(), getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - AccessControlProtos.GetUserPermissionsRequest req = - ShadedAccessControlUtil.buildGetUserPermissionsRequest(getUserPermissionsRequest); - AccessControlProtos.GetUserPermissionsResponse response = - this.master.getUserPermissions(getRpcController(), req); - return response.getUserPermissionList().stream() - .map(userPermission -> ShadedAccessControlUtil.toUserPermission(userPermission)) - .collect(Collectors.toList()); - } - }); - } - - @Override - public List hasUserPermissions(String userName, List permissions) - throws IOException { - return executeCallable( - new MasterCallable>(getConnection(), getRpcControllerFactory()) { - @Override - protected List rpcCall() throws Exception { - HasUserPermissionsRequest request = - ShadedAccessControlUtil.buildHasUserPermissionsRequest(userName, permissions); - return this.master.hasUserPermissions(getRpcController(), request) - .getHasUserPermissionList(); - } - }); - } - - @Override - public void close() { - } - - @Override - public Future splitRegionAsync(byte[] regionName) throws IOException { - return splitRegionAsync(regionName, null); - } - - @Override - public List getRegionMetrics(ServerName serverName) throws IOException { - return getRegionMetrics(serverName, null); - } - - @Override - public Future createTableAsync(TableDescriptor desc) throws IOException { - return createTableAsync(desc, null); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java deleted file mode 100644 index 8806a3e3f28f..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java +++ /dev/null @@ -1,117 +0,0 @@ -/** -* -* 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.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * An implementation of {@link RegionLocator}. Used to view region location information for a single - * HBase table. Lightweight. Get as needed and just close when done. Instances of this class SHOULD - * NOT be constructed directly. Obtain an instance via {@link Connection}. See - * {@link ConnectionFactory} class comment for an example of how. - *

- * This class is thread safe - */ -@InterfaceAudience.Private -public class HRegionLocator implements RegionLocator { - - private final TableName tableName; - private final ConnectionImplementation connection; - - public HRegionLocator(TableName tableName, ConnectionImplementation connection) { - this.connection = connection; - this.tableName = tableName; - } - - /** - * {@inheritDoc} - */ - @Override - public void close() throws IOException { - // This method is required by the RegionLocator interface. This implementation does not have any - // persistent state, so there is no need to do anything here. - } - - @Override - public HRegionLocation getRegionLocation(byte[] row, int replicaId, boolean reload) - throws IOException { - return connection.locateRegion(tableName, row, !reload, true, replicaId) - .getRegionLocation(replicaId); - } - - @Override - public List getRegionLocations(byte[] row, boolean reload) throws IOException { - RegionLocations locs = - connection.locateRegion(tableName, row, !reload, true, RegionInfo.DEFAULT_REPLICA_ID); - return Arrays.asList(locs.getRegionLocations()); - } - - @Override - public List getAllRegionLocations() throws IOException { - ArrayList regions = new ArrayList<>(); - for (RegionLocations locations : listRegionLocations()) { - for (HRegionLocation location : locations.getRegionLocations()) { - regions.add(location); - } - connection.cacheLocation(tableName, locations); - } - return regions; - } - - @Override - public void clearRegionLocationCache() { - connection.clearRegionCache(tableName); - } - - @Override - public TableName getName() { - return this.tableName; - } - - private List listRegionLocations() throws IOException { - if (TableName.isMetaTableName(tableName)) { - return Collections - .singletonList(connection.locateRegion(tableName, HConstants.EMPTY_START_ROW, false, true)); - } - final List regions = new ArrayList<>(); - MetaTableAccessor.Visitor visitor = new MetaTableAccessor.TableVisitorBase(tableName) { - @Override - public boolean visitInternal(Result result) throws IOException { - RegionLocations locations = MetaTableAccessor.getRegionLocations(result); - if (locations == null) { - return true; - } - regions.add(locations); - return true; - } - }; - MetaTableAccessor.scanMetaForTableRegions(connection, visitor, tableName); - return regions; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java deleted file mode 100644 index 11ec3bbc984a..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ /dev/null @@ -1,1103 +0,0 @@ -/** - * - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies; - -// DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY. -// Internally, we use shaded protobuf. This below are part of our public API. -//SEE ABOVE NOTE! -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.Service; -import com.google.protobuf.ServiceException; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; -import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.io.TimeRange; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.ReflectionUtils; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType; - -/** - * An implementation of {@link Table}. Used to communicate with a single HBase table. - * Lightweight. Get as needed and just close when done. - * Instances of this class SHOULD NOT be constructed directly. - * Obtain an instance via {@link Connection}. See {@link ConnectionFactory} - * class comment for an example of how. - * - *

This class is thread safe since 2.0.0 if not invoking any of the setter methods. - * All setters are moved into {@link TableBuilder} and reserved here only for keeping - * backward compatibility, and TODO will be removed soon. - * - *

HTable is no longer a client API. Use {@link Table} instead. It is marked - * InterfaceAudience.Private indicating that this is an HBase-internal class as defined in - * Hadoop - * Interface Classification - * There are no guarantees for backwards source / binary compatibility and methods or class can - * change or go away without deprecation. - * - * @see Table - * @see Admin - * @see Connection - * @see ConnectionFactory - */ -@InterfaceAudience.Private -@InterfaceStability.Stable -class HTable implements Table { - private static final Logger LOG = LoggerFactory.getLogger(HTable.class); - private static final Consistency DEFAULT_CONSISTENCY = Consistency.STRONG; - private final ConnectionImplementation connection; - private final TableName tableName; - private final Configuration configuration; - private final ConnectionConfiguration connConfiguration; - private boolean closed = false; - private final int scannerCaching; - private final long scannerMaxResultSize; - private final ExecutorService pool; // For Multi & Scan - private int operationTimeoutMs; // global timeout for each blocking method with retrying rpc - private final int rpcTimeoutMs; // FIXME we should use this for rpc like batch and checkAndXXX - private int readRpcTimeoutMs; // timeout for each read rpc request - private int writeRpcTimeoutMs; // timeout for each write rpc request - private final boolean cleanupPoolOnClose; // shutdown the pool in close() - private final HRegionLocator locator; - - /** The Async process for batch */ - @VisibleForTesting - AsyncProcess multiAp; - private final RpcRetryingCallerFactory rpcCallerFactory; - private final RpcControllerFactory rpcControllerFactory; - - // Marked Private @since 1.0 - @InterfaceAudience.Private - public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) { - int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE); - if (maxThreads == 0) { - maxThreads = 1; // is there a better default? - } - int corePoolSize = conf.getInt("hbase.htable.threads.coresize", 1); - long keepAliveTime = conf.getLong("hbase.htable.threads.keepalivetime", 60); - - // Using the "direct handoff" approach, new threads will only be created - // if it is necessary and will grow unbounded. This could be bad but in HCM - // we only create as many Runnables as there are region servers. It means - // it also scales when new region servers are added. - ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime, - TimeUnit.SECONDS, new SynchronousQueue<>(), Threads.newDaemonThreadFactory("htable")); - pool.allowCoreThreadTimeOut(true); - return pool; - } - - /** - * Creates an object to access a HBase table. - * Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to - * get a {@link Table} instance (use {@link Table} instead of {@link HTable}). - * @param connection Connection to be used. - * @param builder The table builder - * @param rpcCallerFactory The RPC caller factory - * @param rpcControllerFactory The RPC controller factory - * @param pool ExecutorService to be used. - */ - protected HTable(final ConnectionImplementation connection, - final TableBuilderBase builder, - final RpcRetryingCallerFactory rpcCallerFactory, - final RpcControllerFactory rpcControllerFactory, - final ExecutorService pool) { - this.connection = Preconditions.checkNotNull(connection, "connection is null"); - this.configuration = connection.getConfiguration(); - this.connConfiguration = connection.getConnectionConfiguration(); - 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; - } - - if (rpcControllerFactory == null) { - this.rpcControllerFactory = RpcControllerFactory.instantiate(configuration); - } else { - this.rpcControllerFactory = rpcControllerFactory; - } - - this.tableName = builder.tableName; - this.operationTimeoutMs = builder.operationTimeout; - this.rpcTimeoutMs = builder.rpcTimeout; - this.readRpcTimeoutMs = builder.readRpcTimeout; - this.writeRpcTimeoutMs = builder.writeRpcTimeout; - this.scannerCaching = connConfiguration.getScannerCaching(); - this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize(); - - // puts need to track errors globally due to how the APIs currently work. - multiAp = this.connection.getAsyncProcess(); - this.locator = new HRegionLocator(tableName, connection); - } - - /** - * @return maxKeyValueSize from configuration. - */ - public static int getMaxKeyValueSize(Configuration conf) { - return conf.getInt(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, -1); - } - - @Override - public Configuration getConfiguration() { - return configuration; - } - - @Override - public TableName getName() { - return tableName; - } - - /** - * INTERNAL Used by unit tests and tools to do low-level - * manipulations. - * @return A Connection instance. - */ - @VisibleForTesting - protected Connection getConnection() { - return this.connection; - } - - @Override - public TableDescriptor getDescriptor() throws IOException { - return HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory, - rpcControllerFactory, operationTimeoutMs, readRpcTimeoutMs); - } - - /** - * Get the corresponding start keys and regions for an arbitrary range of - * keys. - *

- * @param startKey Starting row in range, inclusive - * @param endKey Ending row in range - * @param includeEndKey true if endRow is inclusive, false if exclusive - * @return A pair of list of start keys and list of HRegionLocations that - * contain the specified range - * @throws IOException if a remote or network exception occurs - */ - private Pair, List> getKeysAndRegionsInRange( - final byte[] startKey, final byte[] endKey, final boolean includeEndKey) - throws IOException { - return getKeysAndRegionsInRange(startKey, endKey, includeEndKey, false); - } - - /** - * Get the corresponding start keys and regions for an arbitrary range of - * keys. - *

- * @param startKey Starting row in range, inclusive - * @param endKey Ending row in range - * @param includeEndKey true if endRow is inclusive, false if exclusive - * @param reload true to reload information or false to use cached information - * @return A pair of list of start keys and list of HRegionLocations that - * contain the specified range - * @throws IOException if a remote or network exception occurs - */ - private Pair, List> getKeysAndRegionsInRange( - final byte[] startKey, final byte[] endKey, final boolean includeEndKey, - final boolean reload) throws IOException { - final boolean endKeyIsEndOfTable = Bytes.equals(endKey,HConstants.EMPTY_END_ROW); - if ((Bytes.compareTo(startKey, endKey) > 0) && !endKeyIsEndOfTable) { - throw new IllegalArgumentException( - "Invalid range: " + Bytes.toStringBinary(startKey) + - " > " + Bytes.toStringBinary(endKey)); - } - List keysInRange = new ArrayList<>(); - List regionsInRange = new ArrayList<>(); - byte[] currentKey = startKey; - do { - HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload); - keysInRange.add(currentKey); - regionsInRange.add(regionLocation); - currentKey = regionLocation.getRegionInfo().getEndKey(); - } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) - && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0 - || (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0))); - return new Pair<>(keysInRange, regionsInRange); - } - - /** - * The underlying {@link HTable} must not be closed. - * {@link Table#getScanner(Scan)} has other usage details. - */ - @Override - public ResultScanner getScanner(Scan scan) throws IOException { - if (scan.getCaching() <= 0) { - scan.setCaching(scannerCaching); - } - if (scan.getMaxResultSize() <= 0) { - scan.setMaxResultSize(scannerMaxResultSize); - } - if (scan.getMvccReadPoint() > 0) { - // it is not supposed to be set by user, clear - scan.resetMvccReadPoint(); - } - Boolean async = scan.isAsyncPrefetch(); - if (async == null) { - async = connConfiguration.isClientScannerAsyncPrefetch(); - } - - if (scan.isReversed()) { - return new ReversedClientScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } else { - if (async) { - return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } else { - return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } - } - } - - /** - * The underlying {@link HTable} must not be closed. - * {@link Table#getScanner(byte[])} has other usage details. - */ - @Override - public ResultScanner getScanner(byte [] family) throws IOException { - Scan scan = new Scan(); - scan.addFamily(family); - return getScanner(scan); - } - - /** - * The underlying {@link HTable} must not be closed. - * {@link Table#getScanner(byte[], byte[])} has other usage details. - */ - @Override - public ResultScanner getScanner(byte [] family, byte [] qualifier) - throws IOException { - Scan scan = new Scan(); - scan.addColumn(family, qualifier); - return getScanner(scan); - } - - @Override - public Result get(final Get get) throws IOException { - return get(get, get.isCheckExistenceOnly()); - } - - private Result get(Get get, final boolean checkExistenceOnly) throws IOException { - // if we are changing settings to the get, clone it. - if (get.isCheckExistenceOnly() != checkExistenceOnly || get.getConsistency() == null) { - get = ReflectionUtils.newInstance(get.getClass(), get); - get.setCheckExistenceOnly(checkExistenceOnly); - if (get.getConsistency() == null){ - get.setConsistency(DEFAULT_CONSISTENCY); - } - } - - if (get.getConsistency() == Consistency.STRONG) { - final Get configuredGet = get; - ClientServiceCallable callable = new ClientServiceCallable(this.connection, getName(), - get.getRow(), this.rpcControllerFactory.newController(), get.getPriority()) { - @Override - protected Result rpcCall() throws Exception { - ClientProtos.GetRequest request = RequestConverter.buildGetRequest( - getLocation().getRegionInfo().getRegionName(), configuredGet); - ClientProtos.GetResponse response = doGet(request); - return response == null? null: - ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - } - }; - return rpcCallerFactory.newCaller(readRpcTimeoutMs).callWithRetries(callable, - this.operationTimeoutMs); - } - - // Call that takes into account the replica - RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas( - rpcControllerFactory, tableName, this.connection, get, pool, - connConfiguration.getRetriesNumber(), operationTimeoutMs, readRpcTimeoutMs, - connConfiguration.getPrimaryCallTimeoutMicroSecond()); - return callable.call(operationTimeoutMs); - } - - @Override - public Result[] get(List gets) throws IOException { - if (gets.size() == 1) { - return new Result[]{get(gets.get(0))}; - } - try { - Object[] r1 = new Object[gets.size()]; - batch((List)gets, r1, readRpcTimeoutMs); - // Translate. - Result [] results = new Result[r1.length]; - int i = 0; - for (Object obj: r1) { - // Batch ensures if there is a failure we get an exception instead - results[i++] = (Result)obj; - } - return results; - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } - } - - @Override - public void batch(final List actions, final Object[] results) - throws InterruptedException, IOException { - int rpcTimeout = writeRpcTimeoutMs; - 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 = readRpcTimeoutMs; - } - batch(actions, results, rpcTimeout); - } - - public void batch(final List actions, final Object[] results, int rpcTimeout) - throws InterruptedException, IOException { - AsyncProcessTask task = AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(actions) - .setResults(results) - .setRpcTimeout(rpcTimeout) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); - AsyncRequestFuture ars = multiAp.submit(task); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); - } - } - - @Override - public void batchCallback( - final List actions, final Object[] results, final Batch.Callback callback) - throws IOException, InterruptedException { - doBatchWithCallback(actions, results, callback, connection, pool, tableName); - } - - public static void doBatchWithCallback(List actions, Object[] results, - Callback callback, Connection connection, ExecutorService pool, TableName tableName) - throws InterruptedIOException, RetriesExhaustedWithDetailsException { - ConnectionImplementation connImpl = (ConnectionImplementation) connection; - int operationTimeout = connImpl.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 = connImpl.getAsyncProcess().submit(task); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); - } - } - - @Override - public void delete(final Delete delete) throws IOException { - ClientServiceCallable callable = - new ClientServiceCallable(this.connection, getName(), delete.getRow(), - this.rpcControllerFactory.newController(), delete.getPriority()) { - @Override - protected Void rpcCall() throws Exception { - MutateRequest request = RequestConverter - .buildMutateRequest(getLocation().getRegionInfo().getRegionName(), delete); - doMutate(request); - return null; - } - }; - rpcCallerFactory.newCaller(this.writeRpcTimeoutMs) - .callWithRetries(callable, this.operationTimeoutMs); - } - - @Override - public void delete(final List deletes) - throws IOException { - Object[] results = new Object[deletes.size()]; - try { - batch(deletes, results, writeRpcTimeoutMs); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } finally { - // TODO: to be consistent with batch put(), do not modify input list - // mutate list so that it is empty for complete success, or contains only failed records - // results are returned in the same order as the requests in list walk the list backwards, - // so we can remove from list without impacting the indexes of earlier members - for (int i = results.length - 1; i>=0; i--) { - // if result is not null, it succeeded - if (results[i] instanceof Result) { - deletes.remove(i); - } - } - } - } - - @Override - public void put(final Put put) throws IOException { - validatePut(put); - ClientServiceCallable callable = - new ClientServiceCallable(this.connection, getName(), put.getRow(), - this.rpcControllerFactory.newController(), put.getPriority()) { - @Override - protected Void rpcCall() throws Exception { - MutateRequest request = - RequestConverter.buildMutateRequest(getLocation().getRegionInfo().getRegionName(), put); - doMutate(request); - return null; - } - }; - rpcCallerFactory. newCaller(this.writeRpcTimeoutMs).callWithRetries(callable, - this.operationTimeoutMs); - } - - @Override - public void put(final List puts) throws IOException { - for (Put put : puts) { - validatePut(put); - } - Object[] results = new Object[puts.size()]; - try { - batch(puts, results, writeRpcTimeoutMs); - } catch (InterruptedException e) { - throw (InterruptedIOException) new InterruptedIOException().initCause(e); - } - } - - @Override - public void mutateRow(final RowMutations rm) throws IOException { - CancellableRegionServerCallable callable = - new CancellableRegionServerCallable(this.connection, getName(), rm.getRow(), - rpcControllerFactory.newController(), writeRpcTimeoutMs, - new RetryingTimeTracker().start(), rm.getMaxPriority()) { - @Override - protected MultiResponse rpcCall() throws Exception { - RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction( - getLocation().getRegionInfo().getRegionName(), rm); - regionMutationBuilder.setAtomic(true); - MultiRequest request = - MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build(); - ClientProtos.MultiResponse response = doMulti(request); - ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0); - if (res.hasException()) { - Throwable ex = ProtobufUtil.toException(res.getException()); - if (ex instanceof IOException) { - throw (IOException) ex; - } - throw new IOException("Failed to mutate row: " + Bytes.toStringBinary(rm.getRow()), ex); - } - return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); - } - }; - AsyncProcessTask task = AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(rm.getMutations()) - .setCallable(callable) - .setRpcTimeout(writeRpcTimeoutMs) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); - AsyncRequestFuture ars = multiAp.submit(task); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); - } - } - - @Override - public Result append(final Append append) throws IOException { - checkHasFamilies(append); - NoncedRegionServerCallable callable = - new NoncedRegionServerCallable(this.connection, getName(), append.getRow(), - this.rpcControllerFactory.newController(), append.getPriority()) { - @Override - protected Result rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), append, getNonceGroup(), getNonce()); - MutateResponse response = doMutate(request); - if (!response.hasResult()) return null; - return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - } - }; - return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs). - callWithRetries(callable, this.operationTimeoutMs); - } - - @Override - public Result increment(final Increment increment) throws IOException { - checkHasFamilies(increment); - NoncedRegionServerCallable callable = - new NoncedRegionServerCallable(this.connection, getName(), increment.getRow(), - this.rpcControllerFactory.newController(), increment.getPriority()) { - @Override - protected Result rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), increment, getNonceGroup(), getNonce()); - MutateResponse response = doMutate(request); - // Should this check for null like append does? - return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - } - }; - return rpcCallerFactory. newCaller(writeRpcTimeoutMs).callWithRetries(callable, - this.operationTimeoutMs); - } - - @Override - public long incrementColumnValue(final byte [] row, final byte [] family, - final byte [] qualifier, final long amount) - throws IOException { - return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL); - } - - @Override - public long incrementColumnValue(final byte [] row, final byte [] family, - final byte [] qualifier, final long amount, final Durability durability) - throws IOException { - NullPointerException npe = null; - if (row == null) { - npe = new NullPointerException("row is null"); - } else if (family == null) { - npe = new NullPointerException("family is null"); - } - if (npe != null) { - throw new IOException( - "Invalid arguments to incrementColumnValue", npe); - } - - NoncedRegionServerCallable callable = - new NoncedRegionServerCallable(this.connection, getName(), row, - this.rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { - @Override - protected Long rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildIncrementRequest( - getLocation().getRegionInfo().getRegionName(), row, family, - qualifier, amount, durability, getNonceGroup(), getNonce()); - MutateResponse response = doMutate(request); - Result result = ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier))); - } - }; - return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs). - callWithRetries(callable, this.operationTimeoutMs); - } - - private boolean doCheckAndPut(final byte[] row, final byte[] family, final byte[] qualifier, - final String opName, final byte[] value, final TimeRange timeRange, final Put put) - throws IOException { - ClientServiceCallable callable = - new ClientServiceCallable(this.connection, getName(), row, - this.rpcControllerFactory.newController(), put.getPriority()) { - @Override - protected Boolean rpcCall() throws Exception { - CompareType compareType = CompareType.valueOf(opName); - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), row, family, qualifier, - new BinaryComparator(value), compareType, timeRange, put); - MutateResponse response = doMutate(request); - return Boolean.valueOf(response.getProcessed()); - } - }; - return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs) - .callWithRetries(callable, this.operationTimeoutMs); - } - - private boolean doCheckAndDelete(final byte[] row, final byte[] family, final byte[] qualifier, - final String opName, final byte[] value, final TimeRange timeRange, final Delete delete) - throws IOException { - CancellableRegionServerCallable callable = - new CancellableRegionServerCallable(this.connection, getName(), row, - this.rpcControllerFactory.newController(), writeRpcTimeoutMs, - new RetryingTimeTracker().start(), delete.getPriority()) { - @Override - protected SingleResponse rpcCall() throws Exception { - CompareType compareType = CompareType.valueOf(opName); - MutateRequest request = RequestConverter - .buildMutateRequest(getLocation().getRegionInfo().getRegionName(), row, family, - qualifier, new BinaryComparator(value), compareType, timeRange, delete); - MutateResponse response = doMutate(request); - return ResponseConverter.getResult(request, response, getRpcControllerCellScanner()); - } - }; - List rows = Collections.singletonList(delete); - Object[] results = new Object[1]; - AsyncProcessTask task = - AsyncProcessTask.newBuilder().setPool(pool).setTableName(tableName).setRowAccess(rows) - .setCallable(callable) - // TODO any better timeout? - .setRpcTimeout(Math.max(readRpcTimeoutMs, writeRpcTimeoutMs)) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).setResults(results).build(); - AsyncRequestFuture ars = multiAp.submit(task); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); - } - return ((SingleResponse.Entry) results[0]).isProcessed(); - } - - @Override - public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) { - return new CheckAndMutateBuilderImpl(row, family); - } - - private boolean doCheckAndMutate(final byte[] row, final byte[] family, final byte[] qualifier, - final String opName, final byte[] value, final TimeRange timeRange, final RowMutations rm) - throws IOException { - CancellableRegionServerCallable callable = - new CancellableRegionServerCallable(connection, getName(), rm.getRow(), - rpcControllerFactory.newController(), writeRpcTimeoutMs, new RetryingTimeTracker().start(), - rm.getMaxPriority()) { - @Override - protected MultiResponse rpcCall() throws Exception { - CompareType compareType = CompareType.valueOf(opName); - MultiRequest request = RequestConverter - .buildMutateRequest(getLocation().getRegionInfo().getRegionName(), row, family, qualifier, - new BinaryComparator(value), compareType, timeRange, rm); - ClientProtos.MultiResponse response = doMulti(request); - ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0); - if (res.hasException()) { - Throwable ex = ProtobufUtil.toException(res.getException()); - if (ex instanceof IOException) { - throw (IOException) ex; - } - throw new IOException( - "Failed to checkAndMutate row: " + Bytes.toStringBinary(rm.getRow()), ex); - } - return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); - } - }; - - /** - * Currently, we use one array to store 'processed' flag which is returned by server. - * 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()]; - AsyncProcessTask task = AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(rm.getMutations()) - .setResults(results) - .setCallable(callable) - // TODO any better timeout? - .setRpcTimeout(Math.max(readRpcTimeoutMs, writeRpcTimeoutMs)) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); - AsyncRequestFuture ars = multiAp.submit(task); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); - } - - return ((Result)results[0]).getExists(); - } - - @Override - public boolean exists(final Get get) throws IOException { - Result r = get(get, true); - assert r.getExists() != null; - return r.getExists(); - } - - @Override - public boolean[] exists(List gets) throws IOException { - if (gets.isEmpty()) return new boolean[]{}; - if (gets.size() == 1) return new boolean[]{exists(gets.get(0))}; - - ArrayList exists = new ArrayList<>(gets.size()); - for (Get g: gets){ - Get ge = new Get(g); - ge.setCheckExistenceOnly(true); - exists.add(ge); - } - - Object[] r1= new Object[exists.size()]; - try { - batch(exists, r1, readRpcTimeoutMs); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } - - // translate. - boolean[] results = new boolean[r1.length]; - int i = 0; - for (Object o : r1) { - // batch ensures if there is a failure we get an exception instead - results[i++] = ((Result)o).getExists(); - } - - return results; - } - - /** - * Process a mixed batch of Get, Put and Delete actions. All actions for a - * RegionServer are forwarded in one RPC call. Queries are executed in parallel. - * - * @param list The collection of actions. - * @param results An empty array, same size as list. If an exception is thrown, - * you can test here for partial results, and to determine which actions - * processed successfully. - * @throws IOException if there are problems talking to META. Per-item - * exceptions are stored in the results array. - */ - public void processBatchCallback( - final List list, final Object[] results, final Batch.Callback callback) - throws IOException, InterruptedException { - this.batchCallback(list, results, callback); - } - - @Override - public void close() throws IOException { - if (this.closed) { - return; - } - 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"); - } - } - this.closed = true; - } - - // validate for well-formedness - private void validatePut(final Put put) throws IllegalArgumentException { - ConnectionUtils.validatePut(put, connConfiguration.getMaxKeyValueSize()); - } - - /** - * The pool is used for mutli requests for this HTable - * @return the pool used for mutli - */ - ExecutorService getPool() { - return this.pool; - } - - /** - * Explicitly clears the region cache to fetch the latest value from META. - * This is a power user function: avoid unless you know the ramifications. - */ - public void clearRegionCache() { - this.connection.clearRegionLocationCache(); - } - - @Override - public CoprocessorRpcChannel coprocessorService(byte[] row) { - return new RegionCoprocessorRpcChannel(connection, tableName, row); - } - - @Override - public void coprocessorService(final Class service, - byte[] startKey, byte[] endKey, final Batch.Call callable, - final Batch.Callback callback) throws ServiceException, Throwable { - // get regions covered by the row range - List keys = getStartKeysInRange(startKey, endKey); - Map> futures = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for (final byte[] r : keys) { - final RegionCoprocessorRpcChannel channel = - new RegionCoprocessorRpcChannel(connection, tableName, r); - Future future = pool.submit(new Callable() { - @Override - public R call() throws Exception { - T instance = - org.apache.hadoop.hbase.protobuf.ProtobufUtil.newServiceStub(service, channel); - R result = callable.call(instance); - byte[] region = channel.getLastRegion(); - if (callback != null) { - callback.update(region, r, result); - } - return result; - } - }); - futures.put(r, future); - } - for (Map.Entry> e : futures.entrySet()) { - try { - e.getValue().get(); - } catch (ExecutionException ee) { - LOG.warn("Error calling coprocessor service " + service.getName() + " for row " - + Bytes.toStringBinary(e.getKey()), ee); - throw ee.getCause(); - } catch (InterruptedException ie) { - throw new InterruptedIOException("Interrupted calling coprocessor service " - + service.getName() + " for row " + Bytes.toStringBinary(e.getKey())).initCause(ie); - } - } - } - - private List getStartKeysInRange(byte[] start, byte[] end) - throws IOException { - if (start == null) { - start = HConstants.EMPTY_START_ROW; - } - if (end == null) { - end = HConstants.EMPTY_END_ROW; - } - return getKeysAndRegionsInRange(start, end, true).getFirst(); - } - - @Override - public long getRpcTimeout(TimeUnit unit) { - return unit.convert(rpcTimeoutMs, TimeUnit.MILLISECONDS); - } - - @Override - public long getReadRpcTimeout(TimeUnit unit) { - return unit.convert(readRpcTimeoutMs, TimeUnit.MILLISECONDS); - } - - @Override - public long getWriteRpcTimeout(TimeUnit unit) { - return unit.convert(writeRpcTimeoutMs, TimeUnit.MILLISECONDS); - } - - @Override - public long getOperationTimeout(TimeUnit unit) { - return unit.convert(operationTimeoutMs, TimeUnit.MILLISECONDS); - } - - @Override - public String toString() { - return tableName + ";" + connection; - } - - @Override - public void batchCoprocessorService( - final Descriptors.MethodDescriptor methodDescriptor, final Message request, - byte[] startKey, byte[] endKey, final R responsePrototype, final Callback callback) - throws ServiceException, Throwable { - - if (startKey == null) { - startKey = HConstants.EMPTY_START_ROW; - } - if (endKey == null) { - endKey = HConstants.EMPTY_END_ROW; - } - // get regions covered by the row range - Pair, List> keysAndRegions = - getKeysAndRegionsInRange(startKey, endKey, true); - List keys = keysAndRegions.getFirst(); - List regions = keysAndRegions.getSecond(); - - // check if we have any calls to make - if (keys.isEmpty()) { - LOG.info("No regions were selected by key range start=" + Bytes.toStringBinary(startKey) + - ", end=" + Bytes.toStringBinary(endKey)); - return; - } - - List execs = new ArrayList<>(keys.size()); - final Map execsByRow = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for (int i = 0; i < keys.size(); i++) { - final byte[] rowKey = keys.get(i); - final byte[] region = regions.get(i).getRegionInfo().getRegionName(); - RegionCoprocessorServiceExec exec = - new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request); - execs.add(exec); - execsByRow.put(rowKey, exec); - } - - // tracking for any possible deserialization errors on success callback - // TODO: it would be better to be able to reuse AsyncProcess.BatchErrors here - final List callbackErrorExceptions = new ArrayList<>(); - final List callbackErrorActions = new ArrayList<>(); - final List callbackErrorServers = new ArrayList<>(); - Object[] results = new Object[execs.size()]; - - AsyncProcess asyncProcess = new AsyncProcess(connection, configuration, - RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()), - 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(readRpcTimeoutMs) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); - AsyncRequestFuture future = asyncProcess.submit(task); - future.waitUntilDone(); - - if (future.hasError()) { - throw future.getErrors(); - } else if (!callbackErrorExceptions.isEmpty()) { - throw new RetriesExhaustedWithDetailsException(callbackErrorExceptions, callbackErrorActions, - callbackErrorServers); - } - } - - @Override - public RegionLocator getRegionLocator() { - return this.locator; - } - - private class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder { - - private final byte[] row; - private final byte[] family; - private byte[] qualifier; - private TimeRange timeRange; - private CompareOperator op; - private byte[] value; - - CheckAndMutateBuilderImpl(byte[] row, byte[] family) { - this.row = Preconditions.checkNotNull(row, "row is null"); - this.family = Preconditions.checkNotNull(family, "family is null"); - } - - @Override - public CheckAndMutateBuilder qualifier(byte[] qualifier) { - this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" + - " an empty byte array, or just do not call this method if you want a null qualifier"); - return this; - } - - @Override - public CheckAndMutateBuilder timeRange(TimeRange timeRange) { - this.timeRange = timeRange; - return this; - } - - @Override - public CheckAndMutateBuilder ifNotExists() { - this.op = CompareOperator.EQUAL; - this.value = null; - return this; - } - - @Override - public CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value) { - this.op = Preconditions.checkNotNull(compareOp, "compareOp is null"); - this.value = Preconditions.checkNotNull(value, "value is null"); - return this; - } - - private void preCheck() { - Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" + - " calling ifNotExists/ifEquals/ifMatches before executing the request"); - } - - @Override - public boolean thenPut(Put put) throws IOException { - validatePut(put); - preCheck(); - return doCheckAndPut(row, family, qualifier, op.name(), value, timeRange, put); - } - - @Override - public boolean thenDelete(Delete delete) throws IOException { - preCheck(); - return doCheckAndDelete(row, family, qualifier, op.name(), value, timeRange, delete); - } - - @Override - public boolean thenMutate(RowMutations mutation) throws IOException { - preCheck(); - return doCheckAndMutate(row, family, qualifier, op.name(), value, timeRange, mutation); - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java deleted file mode 100644 index 04da2ebe4a5b..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java +++ /dev/null @@ -1,145 +0,0 @@ -/** - * 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.Closeable; -import java.io.IOException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.yetus.audience.InterfaceAudience; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; - -/** - * A RetryingCallable for Master RPC operations. - * Implement the #rpcCall method. It will be retried on error. See its javadoc and the javadoc of - * #call(int). See {@link HBaseAdmin} for examples of how this is used. To get at the - * rpcController that has been created and configured to make this rpc call, use getRpcController(). - * We are trying to contain all protobuf references including references to rpcController so we - * don't pollute codebase with protobuf references; keep the protobuf references contained and only - * present in a few classes rather than all about the code base. - *

Like {@link RegionServerCallable} only in here, we can safely be PayloadCarryingRpcController - * all the time. This is not possible in the similar {@link RegionServerCallable} Callable because - * it has to deal with Coprocessor Endpoints. - * @param return type - */ -@InterfaceAudience.Private -abstract class MasterCallable implements RetryingCallable, Closeable { - protected final ConnectionImplementation connection; - protected MasterKeepAliveConnection master; - private final HBaseRpcController rpcController; - - MasterCallable(ConnectionImplementation connection, - final RpcControllerFactory rpcConnectionFactory) { - this.connection = connection; - this.rpcController = rpcConnectionFactory.newController(); - } - - @Override - public void prepare(boolean reload) throws IOException { - this.master = this.connection.getMaster(); - } - - @Override - public void close() throws IOException { - // The above prepare could fail but this would still be called though masterAdmin is null - if (this.master != null) { - this.master.close(); - this.master = null; - } - } - - @Override - public void throwable(Throwable t, boolean retrying) { - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return ""; - } - - @Override - public long sleep(long pause, int tries) { - return ConnectionUtils.getPauseTime(pause, tries); - } - - /** - * Override that changes the {@link java.util.concurrent.Callable#call()} Exception from {@link Exception} to - * {@link IOException}. It also does setup of an rpcController and calls through to the rpcCall() - * method which callers are expected to implement. If rpcController is an instance of - * PayloadCarryingRpcController, we will set a timeout on it. - */ - @Override - // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate - // and so we contain references to protobuf. We can't set priority on the rpcController as - // we do in RegionServerCallable because we don't always have a Table when we call. - public V call(int callTimeout) throws IOException { - try { - if (this.rpcController != null) { - this.rpcController.reset(); - this.rpcController.setCallTimeout(callTimeout); - } - return rpcCall(); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - /** - * Run the RPC call. Implement this method. To get at the rpcController that has been created - * and configured to make this rpc call, use getRpcController(). We are trying to contain - * rpcController references so we don't pollute codebase with protobuf references; keep the - * protobuf references contained and only present in a few classes rather than all about the - * code base. - * @throws Exception - */ - protected abstract V rpcCall() throws Exception; - - HBaseRpcController getRpcController() { - return this.rpcController; - } - - void setPriority(final int priority) { - if (this.rpcController != null) { - this.rpcController.setPriority(priority); - } - } - - void setPriority(final TableName tableName) { - if (this.rpcController != null) { - this.rpcController.setPriority(tableName); - } - } - - /** - * @param regionName RegionName. If hbase:meta, we'll set high priority. - */ - void setPriority(final byte[] regionName) { - if (isMetaRegion(regionName)) { - setPriority(TableName.META_TABLE_NAME); - } - } - - private static boolean isMetaRegion(final byte[] regionName) { - return Bytes.equals(regionName, RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) || - Bytes.equals(regionName, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes()); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java deleted file mode 100644 index b1c37776f9e4..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * 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.hbase.shaded.protobuf.generated.MasterProtos; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * A KeepAlive connection is not physically closed immediately after the close, - * but rather kept alive for a few minutes. It makes sense only if it is shared. - * - *

This interface is implemented on a stub. It allows to have a #close function in a master - * client. - * - *

This class is intended to be used internally by HBase classes that need to make invocations - * against the master on the MasterProtos.MasterService.BlockingInterface; but not by - * final user code. Hence it's package protected. - */ -@InterfaceAudience.Private -interface MasterKeepAliveConnection extends MasterProtos.MasterService.BlockingInterface { - // Do this instead of implement Closeable because closeable returning IOE is PITA. - void close(); -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java deleted file mode 100644 index fde2838acc26..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaCache.java +++ /dev/null @@ -1,436 +0,0 @@ -/** - * 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 static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent; - -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.CopyOnWriteArraySet; - -import org.apache.hadoop.hbase.HConstants; -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.types.CopyOnWriteArrayMap; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A cache implementation for region locations from meta. - */ -@InterfaceAudience.Private -public class MetaCache { - - private static final Logger LOG = LoggerFactory.getLogger(MetaCache.class); - - /** - * Map of table to table {@link HRegionLocation}s. - */ - private final ConcurrentMap> - cachedRegionLocations = new CopyOnWriteArrayMap<>(); - - // The presence of a server in the map implies it's likely that there is an - // entry in cachedRegionLocations that map to this server; but the absence - // of a server in this map guarantees that there is no entry in cache that - // maps to the absent server. - // The access to this attribute must be protected by a lock on cachedRegionLocations - private final Set cachedServers = new CopyOnWriteArraySet<>(); - - private final MetricsConnection metrics; - - public MetaCache(MetricsConnection metrics) { - this.metrics = metrics; - } - - /** - * Search the cache for a location that fits our table and row key. - * Return null if no suitable region is located. - * - * @return Null or region location found in cache. - */ - public RegionLocations getCachedLocation(final TableName tableName, final byte [] row) { - ConcurrentNavigableMap tableLocations = - getTableLocations(tableName); - - Entry e = tableLocations.floorEntry(row); - if (e == null) { - if (metrics != null) metrics.incrMetaCacheMiss(); - return null; - } - RegionLocations possibleRegion = e.getValue(); - - // make sure that the end key is greater than the row we're looking - // for, otherwise the row actually belongs in the next region, not - // this one. the exception case is when the endkey is - // HConstants.EMPTY_END_ROW, signifying that the region we're - // checking is actually the last region in the table. - byte[] endKey = possibleRegion.getRegionLocation().getRegion().getEndKey(); - // Here we do direct Bytes.compareTo and not doing CellComparator/MetaCellComparator path. - // MetaCellComparator is for comparing against data in META table which need special handling. - // Not doing that is ok for this case because - // 1. We are getting the Region location for the given row in non META tables only. The compare - // checks the given row is within the end key of the found region. So META regions are not - // coming in here. - // 2. Even if META region comes in, its end key will be empty byte[] and so Bytes.equals(endKey, - // HConstants.EMPTY_END_ROW) check itself will pass. - if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) || - Bytes.compareTo(endKey, 0, endKey.length, row, 0, row.length) > 0) { - if (metrics != null) metrics.incrMetaCacheHit(); - return possibleRegion; - } - - // Passed all the way through, so we got nothing - complete cache miss - if (metrics != null) metrics.incrMetaCacheMiss(); - return null; - } - - /** - * Put a newly discovered HRegionLocation into the cache. - * @param tableName The table name. - * @param source the source of the new location - * @param location the new location - */ - public void cacheLocation(final TableName tableName, final ServerName source, - final HRegionLocation location) { - assert source != null; - byte [] startKey = location.getRegion().getStartKey(); - ConcurrentMap tableLocations = getTableLocations(tableName); - RegionLocations locations = new RegionLocations(new HRegionLocation[] {location}) ; - RegionLocations oldLocations = tableLocations.putIfAbsent(startKey, locations); - boolean isNewCacheEntry = (oldLocations == null); - if (isNewCacheEntry) { - if (LOG.isTraceEnabled()) { - LOG.trace("Cached location: " + location); - } - addToCachedServers(locations); - return; - } - - // If the server in cache sends us a redirect, assume it's always valid. - HRegionLocation oldLocation = oldLocations.getRegionLocation( - location.getRegion().getReplicaId()); - boolean force = oldLocation != null && oldLocation.getServerName() != null - && oldLocation.getServerName().equals(source); - - // For redirect if the number is equal to previous - // record, the most common case is that first the region was closed with seqNum, and then - // opened with the same seqNum; hence we will ignore the redirect. - // There are so many corner cases with various combinations of opens and closes that - // an additional counter on top of seqNum would be necessary to handle them all. - RegionLocations updatedLocations = oldLocations.updateLocation(location, false, force); - if (oldLocations != updatedLocations) { - boolean replaced = tableLocations.replace(startKey, oldLocations, updatedLocations); - if (replaced && LOG.isTraceEnabled()) { - LOG.trace("Changed cached location to: " + location); - } - addToCachedServers(updatedLocations); - } - } - - /** - * Put a newly discovered HRegionLocation into the cache. - * @param tableName The table name. - * @param locations the new locations - */ - public void cacheLocation(final TableName tableName, final RegionLocations locations) { - byte [] startKey = locations.getRegionLocation().getRegion().getStartKey(); - ConcurrentMap tableLocations = getTableLocations(tableName); - RegionLocations oldLocation = tableLocations.putIfAbsent(startKey, locations); - boolean isNewCacheEntry = (oldLocation == null); - if (isNewCacheEntry) { - if (LOG.isTraceEnabled()) { - LOG.trace("Cached location: " + locations); - } - addToCachedServers(locations); - return; - } - - // merge old and new locations and add it to the cache - // Meta record might be stale - some (probably the same) server has closed the region - // with later seqNum and told us about the new location. - RegionLocations mergedLocation = oldLocation.mergeLocations(locations); - boolean replaced = tableLocations.replace(startKey, oldLocation, mergedLocation); - if (replaced && LOG.isTraceEnabled()) { - LOG.trace("Merged cached locations: " + mergedLocation); - } - addToCachedServers(locations); - } - - private void addToCachedServers(RegionLocations locations) { - for (HRegionLocation loc : locations.getRegionLocations()) { - if (loc != null) { - cachedServers.add(loc.getServerName()); - } - } - } - - /** - * @param tableName - * @return Map of cached locations for passed tableName - */ - private ConcurrentNavigableMap getTableLocations( - final TableName tableName) { - // find the map of cached locations for this table - return computeIfAbsent(cachedRegionLocations, tableName, - () -> new CopyOnWriteArrayMap<>(Bytes.BYTES_COMPARATOR)); - } - - /** - * Check the region cache to see whether a region is cached yet or not. - * @param tableName tableName - * @param row row - * @return Region cached or not. - */ - public boolean isRegionCached(TableName tableName, final byte[] row) { - RegionLocations location = getCachedLocation(tableName, row); - return location != null; - } - - /** - * Return the number of cached region for a table. It will only be called - * from a unit test. - */ - public int getNumberOfCachedRegionLocations(final TableName tableName) { - Map tableLocs = this.cachedRegionLocations.get(tableName); - if (tableLocs == null) { - return 0; - } - int numRegions = 0; - for (RegionLocations tableLoc : tableLocs.values()) { - numRegions += tableLoc.numNonNullElements(); - } - return numRegions; - } - - /** - * Delete all cached entries. - */ - public void clearCache() { - this.cachedRegionLocations.clear(); - this.cachedServers.clear(); - } - - /** - * Delete all cached entries of a server. - */ - public void clearCache(final ServerName serverName) { - if (!this.cachedServers.contains(serverName)) { - return; - } - - boolean deletedSomething = false; - synchronized (this.cachedServers) { - // We block here, because if there is an error on a server, it's likely that multiple - // threads will get the error simultaneously. If there are hundreds of thousand of - // region location to check, it's better to do this only once. A better pattern would - // be to check if the server is dead when we get the region location. - if (!this.cachedServers.contains(serverName)) { - return; - } - for (ConcurrentMap tableLocations : cachedRegionLocations.values()){ - for (Entry e : tableLocations.entrySet()) { - RegionLocations regionLocations = e.getValue(); - if (regionLocations != null) { - RegionLocations updatedLocations = regionLocations.removeByServer(serverName); - if (updatedLocations != regionLocations) { - if (updatedLocations.isEmpty()) { - deletedSomething |= tableLocations.remove(e.getKey(), regionLocations); - } else { - deletedSomething |= tableLocations.replace(e.getKey(), regionLocations, - updatedLocations); - } - } - } - } - } - this.cachedServers.remove(serverName); - } - if (deletedSomething) { - if (metrics != null) { - metrics.incrMetaCacheNumClearServer(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Removed all cached region locations that map to " + serverName); - } - } - } - - /** - * Delete all cached entries of a table. - */ - public void clearCache(final TableName tableName) { - if (LOG.isTraceEnabled()) { - LOG.trace("Removed all cached region locations for table " + tableName); - } - this.cachedRegionLocations.remove(tableName); - } - - /** - * Delete a cached location, no matter what it is. Called when we were told to not use cache. - * @param tableName tableName - * @param row - */ - public void clearCache(final TableName tableName, final byte [] row) { - ConcurrentMap tableLocations = getTableLocations(tableName); - - RegionLocations regionLocations = getCachedLocation(tableName, row); - if (regionLocations != null) { - byte[] startKey = regionLocations.getRegionLocation().getRegion().getStartKey(); - boolean removed = tableLocations.remove(startKey, regionLocations); - if (removed) { - if (metrics != null) { - metrics.incrMetaCacheNumClearRegion(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Removed " + regionLocations + " from cache"); - } - } - } - } - - /** - * Delete a cached location with specific replicaId. - * @param tableName tableName - * @param row row key - * @param replicaId region replica id - */ - public void clearCache(final TableName tableName, final byte [] row, int replicaId) { - ConcurrentMap tableLocations = getTableLocations(tableName); - - RegionLocations regionLocations = getCachedLocation(tableName, row); - if (regionLocations != null) { - HRegionLocation toBeRemoved = regionLocations.getRegionLocation(replicaId); - if (toBeRemoved != null) { - RegionLocations updatedLocations = regionLocations.remove(replicaId); - byte[] startKey = regionLocations.getRegionLocation().getRegion().getStartKey(); - boolean removed; - if (updatedLocations.isEmpty()) { - removed = tableLocations.remove(startKey, regionLocations); - } else { - removed = tableLocations.replace(startKey, regionLocations, updatedLocations); - } - - if (removed) { - if (metrics != null) { - metrics.incrMetaCacheNumClearRegion(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Removed " + toBeRemoved + " from cache"); - } - } - } - } - } - - /** - * Delete a cached location for a table, row and server - */ - public void clearCache(final TableName tableName, final byte [] row, ServerName serverName) { - ConcurrentMap tableLocations = getTableLocations(tableName); - - RegionLocations regionLocations = getCachedLocation(tableName, row); - if (regionLocations != null) { - RegionLocations updatedLocations = regionLocations.removeByServer(serverName); - if (updatedLocations != regionLocations) { - byte[] startKey = regionLocations.getRegionLocation().getRegion().getStartKey(); - boolean removed = false; - if (updatedLocations.isEmpty()) { - removed = tableLocations.remove(startKey, regionLocations); - } else { - removed = tableLocations.replace(startKey, regionLocations, updatedLocations); - } - if (removed) { - if (metrics != null) { - metrics.incrMetaCacheNumClearRegion(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Removed locations of table: " + tableName + " ,row: " + Bytes.toString(row) - + " mapping to server: " + serverName + " from cache"); - } - } - } - } - } - - /** - * Deletes the cached location of the region if necessary, based on some error from source. - * @param hri The region in question. - */ - public void clearCache(RegionInfo hri) { - ConcurrentMap tableLocations = getTableLocations(hri.getTable()); - RegionLocations regionLocations = tableLocations.get(hri.getStartKey()); - if (regionLocations != null) { - HRegionLocation oldLocation = regionLocations.getRegionLocation(hri.getReplicaId()); - if (oldLocation == null) return; - RegionLocations updatedLocations = regionLocations.remove(oldLocation); - boolean removed; - if (updatedLocations != regionLocations) { - if (updatedLocations.isEmpty()) { - removed = tableLocations.remove(hri.getStartKey(), regionLocations); - } else { - removed = tableLocations.replace(hri.getStartKey(), regionLocations, updatedLocations); - } - if (removed) { - if (metrics != null) { - metrics.incrMetaCacheNumClearRegion(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Removed " + oldLocation + " from cache"); - } - } - } - } - } - - public void clearCache(final HRegionLocation location) { - if (location == null) { - return; - } - TableName tableName = location.getRegion().getTable(); - ConcurrentMap tableLocations = getTableLocations(tableName); - RegionLocations regionLocations = tableLocations.get(location.getRegion().getStartKey()); - if (regionLocations != null) { - RegionLocations updatedLocations = regionLocations.remove(location); - boolean removed; - if (updatedLocations != regionLocations) { - if (updatedLocations.isEmpty()) { - removed = tableLocations.remove(location.getRegion().getStartKey(), regionLocations); - } else { - removed = tableLocations.replace(location.getRegion().getStartKey(), regionLocations, - updatedLocations); - } - if (removed) { - if (metrics != null) { - metrics.incrMetaCacheNumClearRegion(); - } - if (LOG.isTraceEnabled()) { - LOG.trace("Removed " + location + " from cache"); - } - } - } - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java index 193402155150..c9ee3c25dc1c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java @@ -30,8 +30,7 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * Container for Actions (i.e. Get, Delete, or Put), which are grouped by - * regionName. Intended to be used with {@link AsyncProcess}. + * Container for Actions (i.e. Get, Delete, or Put), which are grouped by regionName. */ @InterfaceAudience.Private public final class MultiAction { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java deleted file mode 100644 index bf557faa230d..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java +++ /dev/null @@ -1,158 +0,0 @@ -/** - * 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.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.hadoop.hbase.CellScannable; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.DoNotRetryIOException; -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.yetus.audience.InterfaceAudience; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; - -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction; - -/** - * Callable that handles the multi method call going against a single - * regionserver; i.e. A RegionServerCallable for the multi call (It is NOT a - * RegionServerCallable that goes against multiple regions). - * @param - */ -@InterfaceAudience.Private -class MultiServerCallable extends CancellableRegionServerCallable { - private MultiAction multiAction; - private boolean cellBlock; - - MultiServerCallable(final ConnectionImplementation connection, final TableName tableName, - final ServerName location, final MultiAction multi, RpcController rpcController, - int rpcTimeout, RetryingTimeTracker tracker, int priority) { - super(connection, tableName, null, rpcController, rpcTimeout, tracker, priority); - this.multiAction = multi; - // RegionServerCallable has HRegionLocation field, but this is a multi-region request. - // Using region info from parent HRegionLocation would be a mistake for this class; so - // we will store the server here, and throw if someone tries to obtain location/regioninfo. - this.location = new HRegionLocation(null, location); - this.cellBlock = isCellBlock(); - } - - public void reset(ServerName location, MultiAction multiAction) { - this.location = new HRegionLocation(null, location); - this.multiAction = multiAction; - this.cellBlock = isCellBlock(); - } - - @Override - protected HRegionLocation getLocation() { - throw new RuntimeException("Cannot get region location for multi-region request"); - } - - @Override - public HRegionInfo getHRegionInfo() { - throw new RuntimeException("Cannot get region info for multi-region request"); - } - - MultiAction getMulti() { - return this.multiAction; - } - - @Override - protected MultiResponse rpcCall() throws Exception { - int countOfActions = this.multiAction.size(); - if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions"); - MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder(); - RegionAction.Builder regionActionBuilder = RegionAction.newBuilder(); - ClientProtos.Action.Builder actionBuilder = ClientProtos.Action.newBuilder(); - MutationProto.Builder mutationBuilder = MutationProto.newBuilder(); - - // Pre-size. Presume at least a KV per Action. There are likely more. - List cells = - (this.cellBlock ? new ArrayList(countOfActions) : null); - - long nonceGroup = multiAction.getNonceGroup(); - - // Map from a created RegionAction to the original index for a RowMutations within - // the original list of actions. This will be used to process the results when there - // is RowMutations in the action list. - Map rowMutationsIndexMap = new HashMap<>(); - // The multi object is a list of Actions by region. Iterate by region. - for (Map.Entry> e: this.multiAction.actions.entrySet()) { - final byte [] regionName = e.getKey(); - final List actions = e.getValue(); - if (this.cellBlock) { - // Send data in cellblocks. - // multiRequestBuilder will be populated with region actions. - // rowMutationsIndexMap will be non-empty after the call if there is RowMutations in the - // action list. - RequestConverter.buildNoDataRegionActions(regionName, actions, cells, multiRequestBuilder, - regionActionBuilder, actionBuilder, mutationBuilder, nonceGroup, rowMutationsIndexMap); - } - else { - // multiRequestBuilder will be populated with region actions. - // rowMutationsIndexMap will be non-empty after the call if there is RowMutations in the - // action list. - RequestConverter.buildRegionActions(regionName, actions, multiRequestBuilder, - regionActionBuilder, actionBuilder, mutationBuilder, nonceGroup, rowMutationsIndexMap); - } - } - - if (cells != null) { - setRpcControllerCellScanner(CellUtil.createCellScanner(cells)); - } - ClientProtos.MultiRequest requestProto = multiRequestBuilder.build(); - ClientProtos.MultiResponse responseProto = getStub().multi(getRpcController(), requestProto); - if (responseProto == null) return null; // Occurs on cancel - return ResponseConverter.getResults(requestProto, rowMutationsIndexMap, responseProto, - getRpcControllerCellScanner()); - } - - /** - * @return True if we should send data in cellblocks. This is an expensive call. Cache the - * result if you can rather than call each time. - */ - private boolean isCellBlock() { - // This is not exact -- the configuration could have changed on us after connection was set up - // but it will do for now. - ConnectionImplementation conn = getConnection(); - return conn.hasCellBlockSupport(); - } - - @Override - public void prepare(boolean reload) throws IOException { - // Use the location we were given in the constructor rather than go look it up. - setStub(getConnection().getClient(this.location.getServerName())); - } - - @VisibleForTesting - ServerName getServerName() { - return location.getServerName(); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryableCallerInterceptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryableCallerInterceptor.java deleted file mode 100644 index 2bae4436383e..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryableCallerInterceptor.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * 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 org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException; - -/** - * Class that acts as a NoOpInterceptor. This class is used in case the - * RetryingCallerInterceptor was not configured correctly or an - * RetryingCallerInterceptor was never configured in the first place. - * - */ -@InterfaceAudience.Private -class NoOpRetryableCallerInterceptor extends RetryingCallerInterceptor { - private static final RetryingCallerInterceptorContext NO_OP_CONTEXT = - new NoOpRetryingInterceptorContext(); - - public NoOpRetryableCallerInterceptor() { - } - - public NoOpRetryableCallerInterceptor(Configuration conf) { - super(); - } - - @Override - public void intercept( - RetryingCallerInterceptorContext abstractRetryingCallerInterceptorContext) - throws PreemptiveFastFailException { - } - - @Override - public void handleFailure(RetryingCallerInterceptorContext context, - Throwable t) throws IOException { - } - - @Override - public void updateFailureInfo(RetryingCallerInterceptorContext context) { - } - - @Override - public RetryingCallerInterceptorContext createEmptyContext() { - return NO_OP_CONTEXT; - } - - @Override - public String toString() { - return "NoOpRetryableCallerInterceptor"; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java deleted file mode 100644 index c726ccda4c72..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoOpRetryingInterceptorContext.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * 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.yetus.audience.InterfaceAudience; - -@InterfaceAudience.Private -class NoOpRetryingInterceptorContext extends RetryingCallerInterceptorContext { - - @Override - public void clear() { - // Do Nothing - } - - @Override - public RetryingCallerInterceptorContext prepare(RetryingCallable callable) { - // Do Nothing - return this; - } - - @Override - public RetryingCallerInterceptorContext prepare(RetryingCallable callable, int tries) { - // Do Nothing - return this; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java deleted file mode 100644 index 69f4f4ae613f..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * 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.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; - -/** - * Implementations make an rpc call against a RegionService via a protobuf Service. - * Implement #rpcCall(RpcController) and then call {@link #call(int)} to - * trigger the rpc. The {@link #call(int)} eventually invokes your - * #rpcCall(RpcController) meanwhile saving you having to write a bunch of - * boilerplate. The {@link #call(int)} implementation is from {@link RpcRetryingCaller} so rpcs are - * retried on fail. - * - *

TODO: this class is actually tied to one region, because most of the paths make use of - * the regioninfo part of location when building requests. The only reason it works for - * multi-region requests (e.g. batch) is that they happen to not use the region parts. - * This could be done cleaner (e.g. having a generic parameter and 2 derived classes, - * RegionCallable and actual RegionServerCallable with ServerName. - * @param the class that the ServerCallable handles - */ -@InterfaceAudience.Private -public abstract class NoncedRegionServerCallable extends ClientServiceCallable { - private final long nonce; - - /** - * @param connection Connection to use. - * @param tableName Table name to which row belongs. - * @param row The row we want in tableName. - */ - public NoncedRegionServerCallable(ConnectionImplementation connection, TableName tableName, - byte[] row, HBaseRpcController rpcController, int priority) { - super(connection, tableName, row, rpcController, priority); - this.nonce = getConnection().getNonceGenerator().newNonce(); - } - - long getNonceGroup() { - return getConnection().getNonceGenerator().getNonceGroup(); - } - - long getNonce() { - return this.nonce; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java deleted file mode 100644 index a8406a452a49..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PreemptiveFastFailInterceptor.java +++ /dev/null @@ -1,350 +0,0 @@ -/** - * 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 static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -import java.io.IOException; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import org.apache.commons.lang3.mutable.MutableBoolean; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; -import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException; -import org.apache.hadoop.hbase.ipc.CallTimeoutException; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.ipc.RemoteException; - -/** - * The concrete {@link RetryingCallerInterceptor} class that implements the preemptive fast fail - * feature. - *

- * The motivation is as follows : In case where a large number of clients try and talk to a - * particular region server in hbase, if the region server goes down due to network problems, we - * might end up in a scenario where the clients would go into a state where they all start to retry. - * This behavior will set off many of the threads in pretty much the same path and they all would be - * sleeping giving rise to a state where the client either needs to create more threads to send new - * requests to other hbase machines or block because the client cannot create anymore threads. - *

- * In most cases the clients might prefer to have a bound on the number of threads that are created - * in order to send requests to hbase. This would mostly result in the client thread starvation. - *

- * To circumvent this problem, the approach that is being taken here under is to let 1 of the many - * threads who are trying to contact the regionserver with connection problems and let the other - * threads get a {@link PreemptiveFastFailException} so that they can move on and take other - * requests. - *

- * This would give the client more flexibility on the kind of action he would want to take in cases - * where the regionserver is down. He can either discard the requests and send a nack upstream - * faster or have an application level retry or buffer the requests up so as to send them down to - * hbase later. - */ -@InterfaceAudience.Private -class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor { - - private static final Logger LOG = LoggerFactory - .getLogger(PreemptiveFastFailInterceptor.class); - - // amount of time to wait before we consider a server to be in fast fail - // mode - protected final long fastFailThresholdMilliSec; - - // Keeps track of failures when we cannot talk to a server. Helps in - // fast failing clients if the server is down for a long time. - protected final ConcurrentMap repeatedFailuresMap = new ConcurrentHashMap<>(); - - // We populate repeatedFailuresMap every time there is a failure. So, to - // keep it from growing unbounded, we garbage collect the failure information - // every cleanupInterval. - protected final long failureMapCleanupIntervalMilliSec; - - protected volatile long lastFailureMapCleanupTimeMilliSec; - - // clear failure Info. Used to clean out all entries. - // A safety valve, in case the client does not exit the - // fast fail mode for any reason. - private long fastFailClearingTimeMilliSec; - - private final ThreadLocal threadRetryingInFastFailMode = new ThreadLocal<>(); - - public PreemptiveFastFailInterceptor(Configuration conf) { - this.fastFailThresholdMilliSec = conf.getLong( - HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS, - HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT); - this.failureMapCleanupIntervalMilliSec = conf.getLong( - HConstants.HBASE_CLIENT_FAILURE_MAP_CLEANUP_INTERVAL_MS, - HConstants.HBASE_CLIENT_FAILURE_MAP_CLEANUP_INTERVAL_MS_DEFAULT); - this.fastFailClearingTimeMilliSec = conf.getLong( - HConstants.HBASE_CLIENT_FAST_FAIL_CLEANUP_MS_DURATION_MS, - HConstants.HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT); - lastFailureMapCleanupTimeMilliSec = EnvironmentEdgeManager.currentTime(); - } - - public void intercept(FastFailInterceptorContext context) - throws PreemptiveFastFailException { - context.setFailureInfo(repeatedFailuresMap.get(context.getServer())); - if (inFastFailMode(context.getServer()) && !currentThreadInFastFailMode()) { - // In Fast-fail mode, all but one thread will fast fail. Check - // if we are that one chosen thread. - context.setRetryDespiteFastFailMode(shouldRetryInspiteOfFastFail(context - .getFailureInfo())); - if (!context.isRetryDespiteFastFailMode()) { // we don't have to retry - LOG.debug("Throwing PFFE : " + context.getFailureInfo() + " tries : " - + context.getTries()); - throw new PreemptiveFastFailException( - context.getFailureInfo().numConsecutiveFailures.get(), - context.getFailureInfo().timeOfFirstFailureMilliSec, - context.getFailureInfo().timeOfLatestAttemptMilliSec, context.getServer(), - context.getGuaranteedClientSideOnly().isTrue()); - } - } - context.setDidTry(true); - } - - public void handleFailure(FastFailInterceptorContext context, - Throwable t) throws IOException { - handleThrowable(t, context.getServer(), - context.getCouldNotCommunicateWithServer(), - context.getGuaranteedClientSideOnly()); - } - - public void updateFailureInfo(FastFailInterceptorContext context) { - updateFailureInfoForServer(context.getServer(), context.getFailureInfo(), - context.didTry(), context.getCouldNotCommunicateWithServer() - .booleanValue(), context.isRetryDespiteFastFailMode()); - } - - /** - * Handles failures encountered when communicating with a server. - * - * Updates the FailureInfo in repeatedFailuresMap to reflect the failure. - * Throws RepeatedConnectException if the client is in Fast fail mode. - * - * @param serverName - * @param t - * - the throwable to be handled. - * @throws PreemptiveFastFailException - */ - @VisibleForTesting - protected void handleFailureToServer(ServerName serverName, Throwable t) { - if (serverName == null || t == null) { - return; - } - long currentTime = EnvironmentEdgeManager.currentTime(); - FailureInfo fInfo = - computeIfAbsent(repeatedFailuresMap, serverName, () -> new FailureInfo(currentTime)); - fInfo.timeOfLatestAttemptMilliSec = currentTime; - fInfo.numConsecutiveFailures.incrementAndGet(); - } - - public void handleThrowable(Throwable t1, ServerName serverName, - MutableBoolean couldNotCommunicateWithServer, - MutableBoolean guaranteedClientSideOnly) throws IOException { - Throwable t2 = ClientExceptionsUtil.translatePFFE(t1); - boolean isLocalException = !(t2 instanceof RemoteException); - - if ((isLocalException && ClientExceptionsUtil.isConnectionException(t2))) { - couldNotCommunicateWithServer.setValue(true); - guaranteedClientSideOnly.setValue(!(t2 instanceof CallTimeoutException)); - handleFailureToServer(serverName, t2); - } - } - - /** - * Occasionally cleans up unused information in repeatedFailuresMap. - * - * repeatedFailuresMap stores the failure information for all remote hosts - * that had failures. In order to avoid these from growing indefinitely, - * occassionallyCleanupFailureInformation() will clear these up once every - * cleanupInterval ms. - */ - protected void occasionallyCleanupFailureInformation() { - long now = System.currentTimeMillis(); - if (!(now > lastFailureMapCleanupTimeMilliSec - + failureMapCleanupIntervalMilliSec)) - return; - - // remove entries that haven't been attempted in a while - // No synchronization needed. It is okay if multiple threads try to - // remove the entry again and again from a concurrent hash map. - StringBuilder sb = new StringBuilder(); - for (Entry entry : repeatedFailuresMap.entrySet()) { - if (now > entry.getValue().timeOfLatestAttemptMilliSec - + failureMapCleanupIntervalMilliSec) { // no recent failures - repeatedFailuresMap.remove(entry.getKey()); - } else if (now > entry.getValue().timeOfFirstFailureMilliSec - + this.fastFailClearingTimeMilliSec) { // been failing for a long - // time - LOG.error(entry.getKey() - + " been failing for a long time. clearing out." - + entry.getValue().toString()); - repeatedFailuresMap.remove(entry.getKey()); - } else { - sb.append(entry.getKey().toString()).append(" failing ") - .append(entry.getValue().toString()).append("\n"); - } - } - if (sb.length() > 0) { - LOG.warn("Preemptive failure enabled for : " + sb.toString()); - } - lastFailureMapCleanupTimeMilliSec = now; - } - - /** - * Checks to see if we are in the Fast fail mode for requests to the server. - * - * If a client is unable to contact a server for more than - * fastFailThresholdMilliSec the client will get into fast fail mode. - * - * @param server - * @return true if the client is in fast fail mode for the server. - */ - private boolean inFastFailMode(ServerName server) { - FailureInfo fInfo = repeatedFailuresMap.get(server); - // if fInfo is null --> The server is considered good. - // If the server is bad, wait long enough to believe that the server is - // down. - return (fInfo != null && - EnvironmentEdgeManager.currentTime() > - (fInfo.timeOfFirstFailureMilliSec + this.fastFailThresholdMilliSec)); - } - - /** - * Checks to see if the current thread is already in FastFail mode for *some* - * server. - * - * @return true, if the thread is already in FF mode. - */ - private boolean currentThreadInFastFailMode() { - return (this.threadRetryingInFastFailMode.get() != null && (this.threadRetryingInFastFailMode - .get().booleanValue() == true)); - } - - /** - * Check to see if the client should try to connnect to the server, inspite of - * knowing that it is in the fast fail mode. - * - * The idea here is that we want just one client thread to be actively trying - * to reconnect, while all the other threads trying to reach the server will - * short circuit. - * - * @param fInfo - * @return true if the client should try to connect to the server. - */ - protected boolean shouldRetryInspiteOfFastFail(FailureInfo fInfo) { - // We believe that the server is down, But, we want to have just one - // client - // actively trying to connect. If we are the chosen one, we will retry - // and not throw an exception. - if (fInfo != null - && fInfo.exclusivelyRetringInspiteOfFastFail.compareAndSet(false, true)) { - MutableBoolean threadAlreadyInFF = this.threadRetryingInFastFailMode - .get(); - if (threadAlreadyInFF == null) { - threadAlreadyInFF = new MutableBoolean(); - this.threadRetryingInFastFailMode.set(threadAlreadyInFF); - } - threadAlreadyInFF.setValue(true); - return true; - } else { - return false; - } - } - - /** - * - * This function updates the Failure info for a particular server after the - * attempt to - * - * @param server - * @param fInfo - * @param couldNotCommunicate - * @param retryDespiteFastFailMode - */ - private void updateFailureInfoForServer(ServerName server, - FailureInfo fInfo, boolean didTry, boolean couldNotCommunicate, - boolean retryDespiteFastFailMode) { - if (server == null || fInfo == null || didTry == false) - return; - - // If we were able to connect to the server, reset the failure - // information. - if (couldNotCommunicate == false) { - LOG.info("Clearing out PFFE for server " + server); - repeatedFailuresMap.remove(server); - } else { - // update time of last attempt - long currentTime = System.currentTimeMillis(); - fInfo.timeOfLatestAttemptMilliSec = currentTime; - - // Release the lock if we were retrying inspite of FastFail - if (retryDespiteFastFailMode) { - fInfo.exclusivelyRetringInspiteOfFastFail.set(false); - threadRetryingInFastFailMode.get().setValue(false); - } - } - - occasionallyCleanupFailureInformation(); - } - - @Override - public void intercept(RetryingCallerInterceptorContext context) - throws PreemptiveFastFailException { - if (context instanceof FastFailInterceptorContext) { - intercept((FastFailInterceptorContext) context); - } - } - - @Override - public void handleFailure(RetryingCallerInterceptorContext context, - Throwable t) throws IOException { - if (context instanceof FastFailInterceptorContext) { - handleFailure((FastFailInterceptorContext) context, t); - } - } - - @Override - public void updateFailureInfo(RetryingCallerInterceptorContext context) { - if (context instanceof FastFailInterceptorContext) { - updateFailureInfo((FastFailInterceptorContext) context); - } - } - - @Override - public RetryingCallerInterceptorContext createEmptyContext() { - return new FastFailInterceptorContext(); - } - - protected boolean isServerInFailureMap(ServerName serverName) { - return this.repeatedFailuresMap.containsKey(serverName); - } - - @Override - public String toString() { - return "PreemptiveFastFailInterceptor"; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java deleted file mode 100644 index 33dfefad0aa0..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java +++ /dev/null @@ -1,195 +0,0 @@ -/** - * 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 org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseIOException; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; -import org.apache.hadoop.hbase.util.Bytes; - -/** - * Similar to RegionServerCallable but for the AdminService interface. This service callable - * assumes a Table and row and thus does region locating similar to RegionServerCallable. - * Works against Admin stub rather than Client stub. - */ -@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD", - justification="stub used by ipc") -@InterfaceAudience.Private -public abstract class RegionAdminServiceCallable implements RetryingCallable { - protected AdminService.BlockingInterface stub; - protected final RpcControllerFactory rpcControllerFactory; - private HBaseRpcController controller = null; - - protected final ConnectionImplementation connection; - protected HRegionLocation location; - protected final TableName tableName; - protected final byte[] row; - protected final int replicaId; - - public RegionAdminServiceCallable(ConnectionImplementation connection, - RpcControllerFactory rpcControllerFactory, TableName tableName, byte[] row) { - this(connection, rpcControllerFactory, null, tableName, row); - } - - public RegionAdminServiceCallable(ConnectionImplementation connection, - RpcControllerFactory rpcControllerFactory, HRegionLocation location, - TableName tableName, byte[] row) { - this(connection, rpcControllerFactory, location, - tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID); - } - - public RegionAdminServiceCallable(ConnectionImplementation connection, - RpcControllerFactory rpcControllerFactory, HRegionLocation location, - TableName tableName, byte[] row, int replicaId) { - this.connection = connection; - this.rpcControllerFactory = rpcControllerFactory; - this.location = location; - this.tableName = tableName; - this.row = row; - this.replicaId = replicaId; - } - - @Override - public void prepare(boolean reload) throws IOException { - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - if (reload || location == null) { - location = getLocation(!reload); - } - if (location == null) { - // With this exception, there will be a retry. - throw new HBaseIOException(getExceptionMessage()); - } - this.setStub(connection.getAdmin(location.getServerName())); - } - - protected void setStub(AdminService.BlockingInterface stub) { - this.stub = stub; - } - - public HRegionLocation getLocation(boolean useCache) throws IOException { - RegionLocations rl = getRegionLocations(connection, tableName, row, useCache, replicaId); - if (rl == null) { - throw new HBaseIOException(getExceptionMessage()); - } - HRegionLocation location = rl.getRegionLocation(replicaId); - if (location == null) { - throw new HBaseIOException(getExceptionMessage()); - } - - return location; - } - - @Override - public void throwable(Throwable t, boolean retrying) { - if (location != null) { - connection.updateCachedLocations(tableName, location.getRegionInfo().getRegionName(), row, - t, location.getServerName()); - } - } - - /** - * @return {@link Connection} instance used by this Callable. - */ - Connection getConnection() { - return this.connection; - } - - //subclasses can override this. - protected String getExceptionMessage() { - return "There is no location" + " table=" + tableName - + " ,replica=" + replicaId + ", row=" + Bytes.toStringBinary(row); - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return null; - } - - @Override - public long sleep(long pause, int tries) { - return ConnectionUtils.getPauseTime(pause, tries); - } - - private static RegionLocations getRegionLocations( - ConnectionImplementation connection, TableName tableName, byte[] row, - boolean useCache, int replicaId) - throws RetriesExhaustedException, DoNotRetryIOException, InterruptedIOException { - RegionLocations rl; - try { - rl = connection.locateRegion(tableName, row, useCache, true, replicaId); - } catch (DoNotRetryIOException e) { - throw e; - } catch (RetriesExhaustedException e) { - throw e; - } catch (InterruptedIOException e) { - throw e; - } catch (IOException e) { - throw new RetriesExhaustedException("Can't get the location", e); - } - if (rl == null) { - throw new RetriesExhaustedException("Can't get the locations"); - } - return rl; - } - - /** - * Override that changes Exception from {@link Exception} to {@link IOException}. It also does - * setup of an rpcController and calls through to the unimplemented - * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation. - */ - @Override - // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate - // and so we contain references to protobuf. We can't set priority on the rpcController as - // we do in RegionServerCallable because we don't always have a Table when we call. - public T call(int callTimeout) throws IOException { - this.controller = rpcControllerFactory.newController(); - this.controller.setPriority(this.tableName); - this.controller.setCallTimeout(callTimeout); - try { - return call(this.controller); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - HBaseRpcController getCurrentPayloadCarryingRpcController() { - return this.controller; - } - - /** - * Run RPC call. - * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a - * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this - * class. - * @throws Exception - */ - protected abstract T call(HBaseRpcController rpcController) throws Exception; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java deleted file mode 100644 index 80371b764aa3..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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 org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse; - -import com.google.protobuf.Descriptors; -import com.google.protobuf.Message; -import com.google.protobuf.RpcController; - -/** - * Provides clients with an RPC connection to call Coprocessor Endpoint - * {@link com.google.protobuf.Service}s - * against a given table region. An instance of this class may be obtained - * by calling {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}, - * but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to - * call the endpoint methods. - * @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[]) - */ -@InterfaceAudience.Private -class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel { - private static final Logger LOG = LoggerFactory.getLogger(RegionCoprocessorRpcChannel.class); - private final TableName table; - private final byte [] row; - private final ConnectionImplementation conn; - private byte[] lastRegion; - private final int operationTimeout; - private final RpcRetryingCallerFactory rpcCallerFactory; - - /** - * Constructor - * @param conn connection to use - * @param table to connect to - * @param row to locate region with - */ - RegionCoprocessorRpcChannel(ConnectionImplementation conn, TableName table, byte[] row) { - this.table = table; - this.row = row; - this.conn = conn; - this.operationTimeout = conn.getConnectionConfiguration().getOperationTimeout(); - this.rpcCallerFactory = conn.getRpcRetryingCallerFactory(); - } - - @Override - protected Message callExecService(final RpcController controller, - final Descriptors.MethodDescriptor method, final Message request, - final Message responsePrototype) - throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("Call: " + method.getName() + ", " + request.toString()); - } - if (row == null) { - throw new NullPointerException("Can't be null!"); - } - ClientServiceCallable callable = - new ClientServiceCallable(this.conn, - this.table, this.row, this.conn.getRpcControllerFactory().newController(), HConstants.PRIORITY_UNSET) { - @Override - protected CoprocessorServiceResponse rpcCall() throws Exception { - byte [] regionName = getLocation().getRegionInfo().getRegionName(); - CoprocessorServiceRequest csr = - CoprocessorRpcUtils.getCoprocessorServiceRequest(method, request, row, regionName); - return getStub().execService(getRpcController(), csr); - } - }; - CoprocessorServiceResponse result = - this.rpcCallerFactory. newCaller().callWithRetries(callable, - operationTimeout); - this.lastRegion = result.getRegion().getValue().toByteArray(); - return CoprocessorRpcUtils.getResponse(result, responsePrototype); - } - - /** - * Get last region this RpcChannel communicated with - * @return region name as byte array - */ - public byte[] getLastRegion() { - return lastRegion; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java deleted file mode 100644 index 009544cfc5a2..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java +++ /dev/null @@ -1,238 +0,0 @@ -/** - * 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 org.apache.hadoop.hbase.CellScanner; -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.TableNotEnabledException; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.yetus.audience.InterfaceAudience; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; - -/** - * Implementations make a RPC call against a RegionService via a protobuf Service. - * Implement rpcCall() and the parent class setClientByServiceName; this latter is where the - * RPC stub gets set (the appropriate protobuf 'Service'/Client). Be sure to make use of the - * RpcController that this instance is carrying via #getRpcController(). - * - *

TODO: this class is actually tied to one region, because most of the paths make use of - * the regioninfo part of location when building requests. The only reason it works for - * multi-region requests (e.g. batch) is that they happen to not use the region parts. - * This could be done cleaner (e.g. having a generic parameter and 2 derived classes, - * RegionCallable and actual RegionServerCallable with ServerName. - * - * @param The class that the ServerCallable handles. - * @param The protocol to use (Admin or Client or even an Endpoint over in MetaTableAccessor). - */ -// TODO: MasterCallable and this Class have a lot in common. UNIFY! -// Public but should be package private only it is used by MetaTableAccessor. FIX!! -@InterfaceAudience.Private -public abstract class RegionServerCallable implements RetryingCallable { - private final ConnectionImplementation connection; - private final TableName tableName; - private final byte[] row; - /** - * Some subclasses want to set their own location. Make it protected. - */ - protected HRegionLocation location; - protected S stub; - - /** - * This is 99% of the time a HBaseRpcController but also used doing Coprocessor Endpoints and in - * this case, it is a ServerRpcControllable which is not a HBaseRpcController. - * Can be null! - */ - protected final RpcController rpcController; - private int priority = HConstants.NORMAL_QOS; - - /** - * @param connection Connection to use. - * @param rpcController Controller to use; can be shaded or non-shaded. - * @param tableName Table name to which row belongs. - * @param row The row we want in tableName. - */ - public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte[] row, - RpcController rpcController) { - this(connection, tableName, row, rpcController, HConstants.NORMAL_QOS); - } - - public RegionServerCallable(ConnectionImplementation connection, TableName tableName, byte[] row, - RpcController rpcController, int priority) { - super(); - this.connection = connection; - this.tableName = tableName; - this.row = row; - this.rpcController = rpcController; - this.priority = priority; - } - - protected RpcController getRpcController() { - return this.rpcController; - } - - protected void setStub(S stub) { - this.stub = stub; - } - - protected S getStub() { - return this.stub; - } - - /** - * Override that changes call Exception from {@link Exception} to {@link IOException}. - * Also does set up of the rpcController. - */ - @Override - public T call(int callTimeout) throws IOException { - try { - // Iff non-null and an instance of a SHADED rpcController, do config! Unshaded -- i.e. - // com.google.protobuf.RpcController or null -- will just skip over this config. - if (getRpcController() != null) { - RpcController shadedRpcController = (RpcController)getRpcController(); - // Do a reset to clear previous states, such as CellScanner. - shadedRpcController.reset(); - if (shadedRpcController instanceof HBaseRpcController) { - HBaseRpcController hrc = (HBaseRpcController)getRpcController(); - // If it is an instance of HBaseRpcController, we can set priority on the controller based - // off the tableName. Set call timeout too. - hrc.setPriority(tableName); - hrc.setPriority(priority); - hrc.setCallTimeout(callTimeout); - } - } - return rpcCall(); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - /** - * Run the RPC call. Implement this method. To get at the rpcController that has been created - * and configured to make this rpc call, use getRpcController(). We are trying to contain - * rpcController references so we don't pollute codebase with protobuf references; keep the - * protobuf references contained and only present in a few classes rather than all about the - * code base. - * @throws Exception - */ - protected abstract T rpcCall() throws Exception; - - /** - * Get the RpcController CellScanner. - * If the RpcController is a HBaseRpcController, which it is in all cases except - * when we are processing Coprocessor Endpoint, then this method returns a reference to the - * CellScanner that the HBaseRpcController is carrying. Do it up here in this Callable - * so we don't have to scatter ugly instanceof tests around the codebase. Will return null - * if called in a Coproccessor Endpoint context. Should never happen. - */ - protected CellScanner getRpcControllerCellScanner() { - return (getRpcController() != null && getRpcController() instanceof HBaseRpcController)? - ((HBaseRpcController)getRpcController()).cellScanner(): null; - } - - protected void setRpcControllerCellScanner(CellScanner cellScanner) { - if (getRpcController() != null && getRpcController() instanceof HBaseRpcController) { - ((HBaseRpcController)this.rpcController).setCellScanner(cellScanner); - } - } - - /** - * @return {@link ConnectionImplementation} instance used by this Callable. - */ - protected ConnectionImplementation getConnection() { - return this.connection; - } - - protected HRegionLocation getLocation() { - return this.location; - } - - protected void setLocation(final HRegionLocation location) { - this.location = location; - } - - public TableName getTableName() { - return this.tableName; - } - - public byte [] getRow() { - return this.row; - } - - protected int getPriority() { return this.priority;} - - @Override - public void throwable(Throwable t, boolean retrying) { - if (location != null) { - getConnection().updateCachedLocations(tableName, location.getRegionInfo().getRegionName(), - row, t, location.getServerName()); - } - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return "row '" + Bytes.toString(row) + "' on table '" + tableName + "' at " + location; - } - - @Override - public long sleep(long pause, int tries) { - return ConnectionUtils.getPauseTime(pause, tries); - } - - /** - * @return the HRegionInfo for the current region - */ - public HRegionInfo getHRegionInfo() { - if (this.location == null) { - return null; - } - return this.location.getRegionInfo(); - } - - @Override - public void prepare(final boolean reload) throws IOException { - // check table state if this is a retry - if (reload && tableName != null && !tableName.equals(TableName.META_TABLE_NAME) - && getConnection().isTableDisabled(tableName)) { - throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled."); - } - try (RegionLocator regionLocator = connection.getRegionLocator(tableName)) { - this.location = regionLocator.getRegionLocation(row); - } - if (this.location == null) { - throw new IOException("Failed to find location, tableName=" + tableName + - ", row=" + Bytes.toString(row) + ", reload=" + reload); - } - setStubByServiceName(this.location.getServerName()); - } - - /** - * Set the RCP client stub - * @param serviceName to get the rpc stub for - * @throws IOException When client could not be created - */ - protected abstract void setStubByServiceName(ServerName serviceName) throws IOException; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java deleted file mode 100644 index 70d32d514089..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java +++ /dev/null @@ -1,273 +0,0 @@ -/** - * - * 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.ArrayList; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.RunnableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import org.apache.hadoop.hbase.trace.TraceUtil; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; - -/** - * A completion service for the RpcRetryingCallerFactory. - * Keeps the list of the futures, and allows to cancel them all. - * This means as well that it can be used for a small set of tasks only. - *
Implementation is not Thread safe. - * - * CompletedTasks is implemented as a queue, the entry is added based on the time order. I.e, - * when the first task completes (whether it is a success or failure), it is added as a first - * entry in the queue, the next completed task is added as a second entry in the queue, ... - * When iterating through the queue, we know it is based on time order. If the first - * completed task succeeds, it is returned. If it is failure, the iteration goes on until it - * finds a success. - */ -@InterfaceAudience.Private -public class ResultBoundedCompletionService { - private static final Logger LOG = LoggerFactory.getLogger(ResultBoundedCompletionService.class); - private final RpcRetryingCallerFactory retryingCallerFactory; - private final Executor executor; - private final QueueingFuture[] tasks; // all the tasks - private final ArrayList completedTasks; // completed tasks - private volatile boolean cancelled = false; - - class QueueingFuture implements RunnableFuture { - private final RetryingCallable future; - private T result = null; - private ExecutionException exeEx = null; - private volatile boolean cancelled = false; - private final int callTimeout; - private final RpcRetryingCaller retryingCaller; - private boolean resultObtained = false; - private final int replicaId; // replica id - - - public QueueingFuture(RetryingCallable future, int callTimeout, int id) { - this.future = future; - this.callTimeout = callTimeout; - this.retryingCaller = retryingCallerFactory.newCaller(); - this.replicaId = id; - } - - @SuppressWarnings("unchecked") - @Override - public void run() { - try { - if (!cancelled) { - result = this.retryingCaller.callWithRetries(future, callTimeout); - resultObtained = true; - } - } catch (Throwable t) { - exeEx = new ExecutionException(t); - } finally { - synchronized (tasks) { - // If this wasn't canceled then store the result. - if (!cancelled) { - completedTasks.add(QueueingFuture.this); - } - - // Notify just in case there was someone waiting and this was canceled. - // That shouldn't happen but better safe than sorry. - tasks.notify(); - } - } - } - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - if (resultObtained || exeEx != null) return false; - retryingCaller.cancel(); - if (future instanceof Cancellable) ((Cancellable)future).cancel(); - cancelled = true; - return true; - } - - @Override - public boolean isCancelled() { - return cancelled; - } - - @Override - public boolean isDone() { - return resultObtained || exeEx != null; - } - - @Override - public T get() throws InterruptedException, ExecutionException { - try { - return get(1000, TimeUnit.DAYS); - } catch (TimeoutException e) { - throw new RuntimeException("You did wait for 1000 days here?", e); - } - } - - @Override - public T get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - synchronized (tasks) { - if (resultObtained) { - return result; - } - if (exeEx != null) { - throw exeEx; - } - unit.timedWait(tasks, timeout); - } - if (resultObtained) { - return result; - } - if (exeEx != null) { - throw exeEx; - } - - throw new TimeoutException("timeout=" + timeout + ", " + unit); - } - - public int getReplicaId() { - return replicaId; - } - - public ExecutionException getExeEx() { - return exeEx; - } - } - - @SuppressWarnings("unchecked") - public ResultBoundedCompletionService( - RpcRetryingCallerFactory retryingCallerFactory, Executor executor, - int maxTasks) { - this.retryingCallerFactory = retryingCallerFactory; - this.executor = executor; - this.tasks = new QueueingFuture[maxTasks]; - this.completedTasks = new ArrayList<>(maxTasks); - } - - - public void submit(RetryingCallable task, int callTimeout, int id) { - QueueingFuture newFuture = new QueueingFuture<>(task, callTimeout, id); - executor.execute(TraceUtil.wrap(newFuture, "ResultBoundedCompletionService.submit")); - tasks[id] = newFuture; - } - - public QueueingFuture take() throws InterruptedException { - synchronized (tasks) { - while (!cancelled && (completedTasks.size() < 1)) tasks.wait(); - } - return completedTasks.get(0); - } - - /** - * Poll for the first completed task whether it is a success or execution exception. - * - * @param timeout - time to wait before it times out - * @param unit - time unit for timeout - */ - public QueueingFuture poll(long timeout, TimeUnit unit) throws InterruptedException { - return pollForSpecificCompletedTask(timeout, unit, 0); - } - - /** - * Poll for the first successfully completed task whose completed order is in startIndex, - * endIndex(exclusive) range - * - * @param timeout - time to wait before it times out - * @param unit - time unit for timeout - * @param startIndex - start index, starting from 0, inclusive - * @param endIndex - end index, exclusive - * - * @return If within timeout time, there is no successfully completed task, return null; If all - * tasks get execution exception, it will throw out the last execution exception, - * otherwise return the first successfully completed task's result. - */ - public QueueingFuture pollForFirstSuccessfullyCompletedTask(long timeout, TimeUnit unit, - int startIndex, int endIndex) - throws InterruptedException, CancellationException, ExecutionException { - - QueueingFuture f; - long start, duration; - for (int i = startIndex; i < endIndex; i ++) { - - start = EnvironmentEdgeManager.currentTime(); - f = pollForSpecificCompletedTask(timeout, unit, i); - duration = EnvironmentEdgeManager.currentTime() - start; - - // Even with operationTimeout less than 0, still loop through the rest as there could - // be other completed tasks before operationTimeout. - timeout -= duration; - - if (f == null) { - return null; - } else if (f.getExeEx() != null) { - // we continue here as we need to loop through all the results. - if (LOG.isDebugEnabled()) { - LOG.debug("Replica " + ((f == null) ? 0 : f.getReplicaId()) + " returns " + - f.getExeEx().getCause()); - } - - if (i == (endIndex - 1)) { - // Rethrow this exception - throw f.getExeEx(); - } - continue; - } - - return f; - } - - // impossible to reach - return null; - } - - /** - * Poll for the Nth completed task (index starts from 0 (the 1st), 1 (the second)...) - * - * @param timeout - time to wait before it times out - * @param unit - time unit for timeout - * @param index - the index(th) completed task, index starting from 0 - */ - private QueueingFuture pollForSpecificCompletedTask(long timeout, TimeUnit unit, int index) - throws InterruptedException { - if (index < 0) { - return null; - } - - synchronized (tasks) { - if (!cancelled && (completedTasks.size() <= index)) unit.timedWait(tasks, timeout); - if (completedTasks.size() <= index) return null; - } - return completedTasks.get(index); - } - - public void cancelAll() { - // Grab the lock on tasks so that cancelled is visible everywhere - synchronized (tasks) { - cancelled = true; - } - for (QueueingFuture future : tasks) { - if (future != null) future.cancel(true); - } - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java deleted file mode 100644 index 601f8b882a5c..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * - * 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 org.apache.yetus.audience.InterfaceAudience; - -/** - * A Callable<T> that will be retried. If {@link #call(int)} invocation throws exceptions, - * we will call {@link #throwable(Throwable, boolean)} with whatever the exception was. - * @param result class from executing this - */ -@InterfaceAudience.Private -public interface RetryingCallable { - /** - * Prepare by setting up any connections to servers, etc., ahead of call invocation. - * TODO: We call prepare before EVERY call. Seems wrong. FIX!!!! - * @param reload Set this to true if need to requery locations - * @throws IOException e - */ - void prepare(final boolean reload) throws IOException; - - /** - * Called when call throws an exception and we are going to retry; take action to - * make it so we succeed on next call (clear caches, do relookup of locations, etc.). - * @param t throwable which was thrown - * @param retrying True if we are in retrying mode (we are not in retrying mode when max - * retries == 1; we ARE in retrying mode if retries > 1 even when we are the - * last attempt) - */ - void throwable(final Throwable t, boolean retrying); - - /** - * @return Some details from the implementation that we would like to add to a terminating - * exception; i.e. a fatal exception is being thrown ending retries and we might like to - * add more implementation-specific detail on to the exception being thrown. - */ - String getExceptionMessageAdditionalDetail(); - - /** - * @param pause time to pause - * @param tries amount of tries until till sleep - * @return Suggestion on how much to sleep between retries - */ - long sleep(final long pause, final int tries); - - /** - * Computes a result, or throws an exception if unable to do so. - * - * @param callTimeout - the time available for this call. 0 for infinite. - * @return computed result - * @throws Exception if unable to compute a result - */ - T call(int callTimeout) throws Exception; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptor.java deleted file mode 100644 index e8be79d19749..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptor.java +++ /dev/null @@ -1,98 +0,0 @@ -/** - * 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 org.apache.yetus.audience.InterfaceAudience; - -/** - * This class is designed to fit into the RetryingCaller class which forms the - * central piece of intelligence for the client side retries for most calls. - * - * One can extend this class and intercept the RetryingCaller and add additional - * logic into the execution of a simple HTable operations like get, delete etc. - * - * Concrete implementations of this calls are supposed to the thread safe. The - * object is used across threads to identify the fast failing threads. - * - * For a concrete use case see {@link PreemptiveFastFailInterceptor} - * - * Example use case : - * try { - * interceptor.intercept - * doAction() - * } catch (Exception e) { - * interceptor.handleFailure - * } finally { - * interceptor.updateFaulireInfo - * } - * - * The {@link RetryingCallerInterceptor} also acts as a factory - * for getting a new {@link RetryingCallerInterceptorContext}. - * - */ - -@InterfaceAudience.Private -abstract class RetryingCallerInterceptor { - - protected RetryingCallerInterceptor() { - // Empty constructor protected for NoOpRetryableCallerInterceptor - } - - /** - * This returns the context object for the current call. - * - * @return context : the context that needs to be used during this call. - */ - public abstract RetryingCallerInterceptorContext createEmptyContext(); - - /** - * Call this function in case we caught a failure during retries. - * - * @param context - * : The context object that we obtained previously. - * @param t - * : The exception that we caught in this particular try - * @throws IOException - */ - public abstract void handleFailure(RetryingCallerInterceptorContext context, - Throwable t) throws IOException; - - /** - * Call this function alongside the actual call done on the callable. - * - * @param abstractRetryingCallerInterceptorContext - * @throws PreemptiveFastFailException - */ - public abstract void intercept( - RetryingCallerInterceptorContext abstractRetryingCallerInterceptorContext) - throws IOException; - - /** - * Call this function to update at the end of the retry. This is not necessary - * to happen. - * - * @param context - */ - public abstract void updateFailureInfo( - RetryingCallerInterceptorContext context); - - @Override - public abstract String toString(); -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java deleted file mode 100644 index 70ef56d6d106..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorContext.java +++ /dev/null @@ -1,67 +0,0 @@ -/** - * 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.yetus.audience.InterfaceAudience; - -/** - * The context object used in the {@link RpcRetryingCaller} to enable - * {@link RetryingCallerInterceptor} to intercept calls. - * {@link RetryingCallerInterceptorContext} is the piece of information unique - * to a retrying call that transfers information from the call into the - * {@link RetryingCallerInterceptor} so that {@link RetryingCallerInterceptor} - * can take appropriate action according to the specific logic - * - */ -@InterfaceAudience.Private -abstract class RetryingCallerInterceptorContext { - protected RetryingCallerInterceptorContext() { - } - - /** - * This function clears the internal state of the context object. - */ - public abstract void clear(); - - /** - * This prepares the context object by populating it with information specific - * to the implementation of the {@link RetryingCallerInterceptor} along with - * which this will be used. - * - * @param callable - * : The {@link RetryingCallable} that contains the information about - * the call that is being made. - * @return A new {@link RetryingCallerInterceptorContext} object that can be - * used for use in the current retrying call - */ - public abstract RetryingCallerInterceptorContext prepare(RetryingCallable callable); - - /** - * Telescopic extension that takes which of the many retries we are currently - * in. - * - * @param callable - * : The {@link RetryingCallable} that contains the information about - * the call that is being made. - * @param tries - * : The retry number that we are currently in. - * @return A new context object that can be used for use in the current - * retrying call - */ - public abstract RetryingCallerInterceptorContext prepare(RetryingCallable callable, int tries); -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorFactory.java deleted file mode 100644 index 838e8fc695fa..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallerInterceptorFactory.java +++ /dev/null @@ -1,82 +0,0 @@ -/** - * - * 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.lang.reflect.Constructor; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Factory implementation to provide the {@link ConnectionImplementation} with - * the implementation of the {@link RetryingCallerInterceptor} that we would use - * to intercept the {@link RpcRetryingCaller} during the course of their calls. - * - */ - -@InterfaceAudience.Private -class RetryingCallerInterceptorFactory { - private static final Logger LOG = LoggerFactory - .getLogger(RetryingCallerInterceptorFactory.class); - private Configuration conf; - private final boolean failFast; - public static final RetryingCallerInterceptor NO_OP_INTERCEPTOR = - new NoOpRetryableCallerInterceptor(null); - - public RetryingCallerInterceptorFactory(Configuration conf) { - this.conf = conf; - failFast = conf.getBoolean(HConstants.HBASE_CLIENT_FAST_FAIL_MODE_ENABLED, - HConstants.HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT); - } - - /** - * This builds the implementation of {@link RetryingCallerInterceptor} that we - * specify in the conf and returns the same. - * - * To use {@link PreemptiveFastFailInterceptor}, set HBASE_CLIENT_ENABLE_FAST_FAIL_MODE to true. - * HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL is defaulted to {@link PreemptiveFastFailInterceptor} - * - * @return The factory build method which creates the - * {@link RetryingCallerInterceptor} object according to the - * configuration. - */ - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", - justification="Convert thrown exception to unchecked") - public RetryingCallerInterceptor build() { - RetryingCallerInterceptor ret = NO_OP_INTERCEPTOR; - if (failFast) { - try { - Class c = conf.getClass( - HConstants.HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL, - PreemptiveFastFailInterceptor.class); - Constructor constructor = c - .getDeclaredConstructor(Configuration.class); - constructor.setAccessible(true); - ret = (RetryingCallerInterceptor) constructor.newInstance(conf); - } catch (Exception e) { - ret = new PreemptiveFastFailInterceptor(conf); - } - } - LOG.trace("Using " + ret.toString() + " for intercepting the RpcRetryingCaller"); - return ret; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java deleted file mode 100644 index e804e9254605..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.hbase.util.EnvironmentEdgeManager; - -/** - * Tracks the amount of time remaining for an operation. - */ -class RetryingTimeTracker { - private long globalStartTime = -1; - - public RetryingTimeTracker start() { - if (this.globalStartTime < 0) { - this.globalStartTime = EnvironmentEdgeManager.currentTime(); - } - return this; - } - - public int getRemainingTime(int callTimeout) { - if (callTimeout <= 0) { - return 0; - } else { - if (callTimeout == Integer.MAX_VALUE) { - return Integer.MAX_VALUE; - } - long remaining = EnvironmentEdgeManager.currentTime() - this.globalStartTime; - long remainingTime = callTimeout - remaining; - if (remainingTime < 1) { - // If there is no time left, we're trying anyway. It's too late. - // 0 means no timeout, and it's not the intent here. So we secure both cases by - // resetting to the minimum. - remainingTime = 1; - } - if (remainingTime > Integer.MAX_VALUE) { - throw new RuntimeException("remainingTime=" + remainingTime + - " which is > Integer.MAX_VALUE"); - } - return (int)remainingTime; - } - } - - public long getStartTime() { - return this.globalStartTime; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java deleted file mode 100644 index 2ed037e8a98c..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan; - -import java.io.IOException; -import java.util.concurrent.ExecutorService; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; - -/** - * A reversed client scanner which support backward scanning - */ -@InterfaceAudience.Private -public class ReversedClientScanner extends ClientScanner { - - /** - * Create a new ReversibleClientScanner for the specified table Note that the passed - * {@link Scan}'s start row maybe changed. - */ - public ReversedClientScanner(Configuration conf, Scan scan, TableName tableName, - ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) - throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - primaryOperationTimeout); - } - - @Override - protected boolean setNewStartKey() { - if (noMoreResultsForReverseScan(scan, currentRegion)) { - return false; - } - scan.withStartRow(currentRegion.getStartKey(), false); - return true; - } - - @Override - protected ReversedScannerCallable createScannerCallable() { - return new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - this.rpcControllerFactory); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java deleted file mode 100644 index 6a325b26a9f8..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java +++ /dev/null @@ -1,166 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.createCloseRowBefore; -import static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCRetriesMetrics; -import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Bytes; - - -/** - * A reversed ScannerCallable which supports backward scanning. - */ -@InterfaceAudience.Private -public class ReversedScannerCallable extends ScannerCallable { - - /** - * @param connection - * @param tableName - * @param scan - * @param scanMetrics - * @param rpcFactory to create an {@link com.google.protobuf.RpcController} to talk to the - * regionserver - */ - public ReversedScannerCallable(ConnectionImplementation connection, TableName tableName, - Scan scan, ScanMetrics scanMetrics, RpcControllerFactory rpcFactory) { - super(connection, tableName, scan, scanMetrics, rpcFactory); - } - - /** - * @param connection - * @param tableName - * @param scan - * @param scanMetrics - * @param rpcFactory to create an {@link com.google.protobuf.RpcController} to talk to the - * regionserver - * @param replicaId the replica id - */ - public ReversedScannerCallable(ConnectionImplementation connection, TableName tableName, - Scan scan, ScanMetrics scanMetrics, RpcControllerFactory rpcFactory, int replicaId) { - super(connection, tableName, scan, scanMetrics, rpcFactory, replicaId); - } - - /** - * @param reload force reload of server location - * @throws IOException - */ - @Override - public void prepare(boolean reload) throws IOException { - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - if (!instantiated || reload) { - // we should use range locate if - // 1. we do not want the start row - // 2. the start row is empty which means we need to locate to the last region. - if (scan.includeStartRow() && !isEmptyStartRow(getRow())) { - // Just locate the region with the row - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(!reload, id, - getConnection(), getTableName(), getRow()); - this.location = id < rl.size() ? rl.getRegionLocation(id) : null; - if (location == null || location.getServerName() == null) { - throw new IOException("Failed to find location, tableName=" - + getTableName() + ", row=" + Bytes.toStringBinary(getRow()) + ", reload=" - + reload); - } - } else { - // Need to locate the regions with the range, and the target location is - // the last one which is the previous region of last region scanner - byte[] locateStartRow = createCloseRowBefore(getRow()); - List locatedRegions = locateRegionsInRange( - locateStartRow, getRow(), reload); - if (locatedRegions.isEmpty()) { - throw new DoNotRetryIOException( - "Does hbase:meta exist hole? Couldn't get regions for the range from " - + Bytes.toStringBinary(locateStartRow) + " to " - + Bytes.toStringBinary(getRow())); - } - this.location = locatedRegions.get(locatedRegions.size() - 1); - } - setStub(getConnection().getClient(getLocation().getServerName())); - checkIfRegionServerIsRemote(); - instantiated = true; - } - - // check how often we retry. - if (reload) { - incRPCRetriesMetrics(scanMetrics, isRegionServerRemote); - } - } - - /** - * Get the corresponding regions for an arbitrary range of keys. - * @param startKey Starting row in range, inclusive - * @param endKey Ending row in range, exclusive - * @param reload force reload of server location - * @return A list of HRegionLocation corresponding to the regions that contain - * the specified range - * @throws IOException - */ - private List locateRegionsInRange(byte[] startKey, - byte[] endKey, boolean reload) throws IOException { - final boolean endKeyIsEndOfTable = Bytes.equals(endKey, - HConstants.EMPTY_END_ROW); - if ((Bytes.compareTo(startKey, endKey) > 0) && !endKeyIsEndOfTable) { - throw new IllegalArgumentException("Invalid range: " - + Bytes.toStringBinary(startKey) + " > " - + Bytes.toStringBinary(endKey)); - } - List regionList = new ArrayList<>(); - byte[] currentKey = startKey; - do { - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(!reload, id, - getConnection(), getTableName(), currentKey); - HRegionLocation regionLocation = id < rl.size() ? rl.getRegionLocation(id) : null; - if (regionLocation != null && regionLocation.getRegionInfo().containsRow(currentKey)) { - regionList.add(regionLocation); - } else { - throw new DoNotRetryIOException("Does hbase:meta exist hole? Locating row " - + Bytes.toStringBinary(currentKey) + " returns incorrect region " - + (regionLocation == null ? null : regionLocation.getRegionInfo())); - } - currentKey = regionLocation.getRegionInfo().getEndKey(); - } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) - && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0)); - return regionList; - } - - @Override - public ScannerCallable getScannerCallableForReplica(int id) { - ReversedScannerCallable r = new ReversedScannerCallable(getConnection(), getTableName(), - this.getScan(), this.scanMetrics, rpcControllerFactory, id); - r.setCaching(this.getCaching()); - return r; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java deleted file mode 100644 index 4f93bd9a26e4..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * 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.Closeable; -import java.io.IOException; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; - -/** - * A RetryingCallable for RPC connection operations. - * @param return type - */ -abstract class RpcRetryingCallable implements RetryingCallable, Closeable { - @Override - public void prepare(boolean reload) throws IOException { - } - - @Override - public void close() throws IOException { - } - - @Override - public void throwable(Throwable t, boolean retrying) { - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return ""; - } - - @Override - public long sleep(long pause, int tries) { - return ConnectionUtils.getPauseTime(pause, tries); - } - - @Override - // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate - // and so we contain references to protobuf. - public V call(int callTimeout) throws IOException { - try { - return rpcCall(callTimeout); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - protected abstract V rpcCall(int callTimeout) throws Exception; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java deleted file mode 100644 index 9d30221cd55d..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * 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.yetus.audience.InterfaceAudience; - -import java.io.IOException; - -@InterfaceAudience.Public -public interface RpcRetryingCaller { - void cancel(); - - /** - * Retries if invocation fails. - * @param callTimeout Timeout for this call - * @param callable The {@link RetryingCallable} to run. - * @return an object of type T - * @throws IOException if a remote or network exception occurs - * @throws RuntimeException other unspecified error - */ - T callWithRetries(RetryingCallable callable, int callTimeout) - throws IOException, RuntimeException; - - /** - * Call the server once only. - * {@link RetryingCallable} has a strange shape so we can do retries. Use this invocation if you - * want to do a single call only (A call to {@link RetryingCallable#call(int)} will not likely - * succeed). - * @return an object of type T - * @throws IOException if a remote or network exception occurs - * @throws RuntimeException other unspecified error - */ - T callWithoutRetries(RetryingCallable callable, int callTimeout) - throws IOException, RuntimeException; -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java deleted file mode 100644 index e7a3d1801044..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerFactory.java +++ /dev/null @@ -1,130 +0,0 @@ -/** - * 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.HConstants; -import org.apache.hadoop.hbase.util.ReflectionUtils; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Factory to create an {@link RpcRetryingCaller} - */ -@InterfaceAudience.Private -public class RpcRetryingCallerFactory { - - /** Configuration key for a custom {@link RpcRetryingCaller} */ - public static final String CUSTOM_CALLER_CONF_KEY = "hbase.rpc.callerfactory.class"; - private static final Logger LOG = LoggerFactory.getLogger(RpcRetryingCallerFactory.class); - protected final Configuration conf; - private final long pause; - private final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified - private final int retries; - private final int rpcTimeout; - private final RetryingCallerInterceptor interceptor; - private final int startLogErrorsCnt; - /* These below data members are UNUSED!!!*/ - private final boolean enableBackPressure; - private ServerStatisticTracker stats; - - public RpcRetryingCallerFactory(Configuration conf) { - this(conf, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR); - } - - public RpcRetryingCallerFactory(Configuration conf, RetryingCallerInterceptor interceptor) { - this.conf = conf; - pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); - if (configuredPauseForCQTBE < pause) { - LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " - + configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE - + ", will use " + pause + " instead."); - this.pauseForCQTBE = pause; - } else { - this.pauseForCQTBE = configuredPauseForCQTBE; - } - retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, - HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); - startLogErrorsCnt = conf.getInt(AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY, - AsyncProcess.DEFAULT_START_LOG_ERRORS_AFTER_COUNT); - this.interceptor = interceptor; - enableBackPressure = conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, - HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE); - rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - } - - /** - * Set the tracker that should be used for tracking statistics about the server - */ - public void setStatisticTracker(ServerStatisticTracker statisticTracker) { - this.stats = statisticTracker; - } - - /** - * Create a new RetryingCaller with specific rpc timeout. - */ - public RpcRetryingCaller newCaller(int rpcTimeout) { - // We store the values in the factory instance. This way, constructing new objects - // is cheap as it does not require parsing a complex structure. - RpcRetryingCaller caller = new RpcRetryingCallerImpl<>(pause, pauseForCQTBE, retries, - interceptor, startLogErrorsCnt, rpcTimeout); - return caller; - } - - /** - * Create a new RetryingCaller with configured rpc timeout. - */ - public RpcRetryingCaller newCaller() { - // We store the values in the factory instance. This way, constructing new objects - // is cheap as it does not require parsing a complex structure. - RpcRetryingCaller caller = new RpcRetryingCallerImpl<>(pause, pauseForCQTBE, retries, - interceptor, startLogErrorsCnt, rpcTimeout); - return caller; - } - - public static RpcRetryingCallerFactory instantiate(Configuration configuration) { - return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null); - } - - public static RpcRetryingCallerFactory instantiate(Configuration configuration, - ServerStatisticTracker stats) { - return instantiate(configuration, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, stats); - } - - public static RpcRetryingCallerFactory instantiate(Configuration configuration, - RetryingCallerInterceptor interceptor, ServerStatisticTracker stats) { - String clazzName = RpcRetryingCallerFactory.class.getName(); - String rpcCallerFactoryClazz = - configuration.get(RpcRetryingCallerFactory.CUSTOM_CALLER_CONF_KEY, clazzName); - RpcRetryingCallerFactory factory; - if (rpcCallerFactoryClazz.equals(clazzName)) { - factory = new RpcRetryingCallerFactory(configuration, interceptor); - } else { - factory = ReflectionUtils.instantiateWithCustomCtor( - rpcCallerFactoryClazz, new Class[] { Configuration.class }, - new Object[] { configuration }); - } - - // setting for backwards compat with existing caller factories, rather than in the ctor - factory.setStatisticTracker(stats); - return factory; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java deleted file mode 100644 index 7a850eeb9d13..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java +++ /dev/null @@ -1,252 +0,0 @@ -/** - * - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.lang.reflect.UndeclaredThrowableException; -import java.net.SocketTimeoutException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.hadoop.hbase.CallQueueTooBigException; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.ExceptionUtil; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.util.StringUtils; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - -/** - * Runs an rpc'ing {@link RetryingCallable}. Sets into rpc client - * threadlocal outstanding timeouts as so we don't persist too much. - * Dynamic rather than static so can set the generic appropriately. - * - * This object has a state. It should not be used by in parallel by different threads. - * Reusing it is possible however, even between multiple threads. However, the user will - * have to manage the synchronization on its side: there is no synchronization inside the class. - */ -@InterfaceAudience.Private -public class RpcRetryingCallerImpl implements RpcRetryingCaller { - // LOG is being used in TestMultiRowRangeFilter, hence leaving it public - public static final Logger LOG = LoggerFactory.getLogger(RpcRetryingCallerImpl.class); - - /** How many retries are allowed before we start to log */ - private final int startLogErrorsCnt; - - private final long pause; - private final long pauseForCQTBE; - private final int maxAttempts;// how many times to try - private final int rpcTimeout;// timeout for each rpc request - private final AtomicBoolean cancelled = new AtomicBoolean(false); - private final RetryingCallerInterceptor interceptor; - private final RetryingCallerInterceptorContext context; - private final RetryingTimeTracker tracker; - - public RpcRetryingCallerImpl(long pause, long pauseForCQTBE, int retries, int startLogErrorsCnt) { - this(pause, pauseForCQTBE, retries, RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, - startLogErrorsCnt, 0); - } - - public RpcRetryingCallerImpl(long pause, long pauseForCQTBE, int retries, - RetryingCallerInterceptor interceptor, int startLogErrorsCnt, int rpcTimeout) { - this.pause = pause; - this.pauseForCQTBE = pauseForCQTBE; - this.maxAttempts = retries2Attempts(retries); - this.interceptor = interceptor; - context = interceptor.createEmptyContext(); - this.startLogErrorsCnt = startLogErrorsCnt; - this.tracker = new RetryingTimeTracker(); - this.rpcTimeout = rpcTimeout; - } - - @Override - public void cancel(){ - cancelled.set(true); - synchronized (cancelled){ - cancelled.notifyAll(); - } - } - - @Override - public T callWithRetries(RetryingCallable callable, int callTimeout) - throws IOException, RuntimeException { - List exceptions = new ArrayList<>(); - tracker.start(); - context.clear(); - for (int tries = 0;; tries++) { - long expectedSleep; - try { - // bad cache entries are cleared in the call to RetryingCallable#throwable() in catch block - callable.prepare(tries != 0); - interceptor.intercept(context.prepare(callable, tries)); - return callable.call(getTimeout(callTimeout)); - } catch (PreemptiveFastFailException e) { - throw e; - } catch (Throwable t) { - ExceptionUtil.rethrowIfInterrupt(t); - Throwable cause = t.getCause(); - if (cause instanceof DoNotRetryIOException) { - // Fail fast - throw (DoNotRetryIOException) cause; - } - // translateException throws exception when should not retry: i.e. when request is bad. - interceptor.handleFailure(context, t); - t = translateException(t); - - if (tries > startLogErrorsCnt) { - if (LOG.isInfoEnabled()) { - StringBuilder builder = new StringBuilder("Call exception, tries=").append(tries) - .append(", retries=").append(maxAttempts).append(", started=") - .append((EnvironmentEdgeManager.currentTime() - tracker.getStartTime())) - .append(" ms ago, ").append("cancelled=").append(cancelled.get()) - .append(", msg=").append(t.getMessage()) - .append(", details=").append(callable.getExceptionMessageAdditionalDetail()) - .append(", see https://s.apache.org/timeout"); - if (LOG.isDebugEnabled()) { - builder.append(", exception=").append(StringUtils.stringifyException(t)); - LOG.debug(builder.toString()); - } else { - LOG.info(builder.toString()); - } - } - } - - callable.throwable(t, maxAttempts != 1); - RetriesExhaustedException.ThrowableWithExtraContext qt = - new RetriesExhaustedException.ThrowableWithExtraContext(t, - EnvironmentEdgeManager.currentTime(), toString()); - exceptions.add(qt); - if (tries >= maxAttempts - 1) { - throw new RetriesExhaustedException(tries, exceptions); - } - // If the server is dead, we need to wait a little before retrying, to give - // a chance to the regions to be moved - // get right pause time, start by RETRY_BACKOFF[0] * pauseBase, where pauseBase might be - // special when encountering CallQueueTooBigException, see #HBASE-17114 - long pauseBase = (t instanceof CallQueueTooBigException) ? pauseForCQTBE : pause; - expectedSleep = callable.sleep(pauseBase, tries); - - // If, after the planned sleep, there won't be enough time left, we stop now. - long duration = singleCallDuration(expectedSleep); - if (duration > callTimeout) { - String msg = "callTimeout=" + callTimeout + ", callDuration=" + duration + - ": " + t.getMessage() + " " + callable.getExceptionMessageAdditionalDetail(); - throw (SocketTimeoutException)(new SocketTimeoutException(msg).initCause(t)); - } - } finally { - interceptor.updateFailureInfo(context); - } - try { - if (expectedSleep > 0) { - synchronized (cancelled) { - if (cancelled.get()) return null; - cancelled.wait(expectedSleep); - } - } - if (cancelled.get()) return null; - } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted after " + tries - + " tries while maxAttempts=" + maxAttempts); - } - } - } - - /** - * @return Calculate how long a single call took - */ - private long singleCallDuration(final long expectedSleep) { - return (EnvironmentEdgeManager.currentTime() - tracker.getStartTime()) + expectedSleep; - } - - @Override - public T callWithoutRetries(RetryingCallable callable, int callTimeout) - throws IOException, RuntimeException { - // The code of this method should be shared with withRetries. - try { - callable.prepare(false); - return callable.call(callTimeout); - } catch (Throwable t) { - Throwable t2 = translateException(t); - ExceptionUtil.rethrowIfInterrupt(t2); - // It would be nice to clear the location cache here. - if (t2 instanceof IOException) { - throw (IOException)t2; - } else { - throw new RuntimeException(t2); - } - } - } - - /** - * Get the good or the remote exception if any, throws the DoNotRetryIOException. - * @param t the throwable to analyze - * @return the translated exception, if it's not a DoNotRetryIOException - * @throws DoNotRetryIOException - if we find it, we throw it instead of translating. - */ - static Throwable translateException(Throwable t) throws DoNotRetryIOException { - if (t instanceof UndeclaredThrowableException) { - if (t.getCause() != null) { - t = t.getCause(); - } - } - if (t instanceof RemoteException) { - t = ((RemoteException)t).unwrapRemoteException(); - } - if (t instanceof LinkageError) { - throw new DoNotRetryIOException(t); - } - if (t instanceof ServiceException) { - ServiceException se = (ServiceException)t; - Throwable cause = se.getCause(); - if (cause != null && cause instanceof DoNotRetryIOException) { - throw (DoNotRetryIOException)cause; - } - // Don't let ServiceException out; its rpc specific. - t = cause; - // t could be a RemoteException so go around again. - translateException(t); - } else if (t instanceof DoNotRetryIOException) { - throw (DoNotRetryIOException)t; - } - return t; - } - - private int getTimeout(int callTimeout){ - int timeout = tracker.getRemainingTime(callTimeout); - if (timeout <= 0 || rpcTimeout > 0 && rpcTimeout < timeout){ - timeout = rpcTimeout; - } - return timeout; - } - - @Override - public String toString() { - return "RpcRetryingCaller{" + "globalStartTime=" + tracker.getStartTime() + - ", pause=" + pause + ", maxAttempts=" + maxAttempts + '}'; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java deleted file mode 100644 index 56985c0e204c..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java +++ /dev/null @@ -1,336 +0,0 @@ -/** - * 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 static org.apache.hadoop.hbase.HConstants.PRIORITY_UNSET; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseIOException; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; - -/** - * Caller that goes to replica if the primary region does no answer within a configurable - * timeout. If the timeout is reached, it calls all the secondary replicas, and returns - * the first answer. If the answer comes from one of the secondary replica, it will - * be marked as stale. - */ -@InterfaceAudience.Private -public class RpcRetryingCallerWithReadReplicas { - private static final Logger LOG = - LoggerFactory.getLogger(RpcRetryingCallerWithReadReplicas.class); - - protected final ExecutorService pool; - protected final ConnectionImplementation cConnection; - protected final Configuration conf; - protected final Get get; - protected final TableName tableName; - protected final int timeBeforeReplicas; - private final int operationTimeout; - private final int rpcTimeout; - private final int retries; - private final RpcControllerFactory rpcControllerFactory; - private final RpcRetryingCallerFactory rpcRetryingCallerFactory; - - public RpcRetryingCallerWithReadReplicas( - RpcControllerFactory rpcControllerFactory, TableName tableName, - ConnectionImplementation cConnection, final Get get, - ExecutorService pool, int retries, int operationTimeout, int rpcTimeout, - int timeBeforeReplicas) { - this.rpcControllerFactory = rpcControllerFactory; - this.tableName = tableName; - this.cConnection = cConnection; - this.conf = cConnection.getConfiguration(); - this.get = get; - this.pool = pool; - this.retries = retries; - this.operationTimeout = operationTimeout; - this.rpcTimeout = rpcTimeout; - this.timeBeforeReplicas = timeBeforeReplicas; - this.rpcRetryingCallerFactory = new RpcRetryingCallerFactory(conf); - } - - /** - * A RegionServerCallable that takes into account the replicas, i.e. - * - the call can be on any replica - * - we need to stop retrying when the call is completed - * - we can be interrupted - */ - class ReplicaRegionServerCallable extends CancellableRegionServerCallable { - final int id; - public ReplicaRegionServerCallable(int id, HRegionLocation location) { - super(RpcRetryingCallerWithReadReplicas.this.cConnection, - RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow(), - rpcControllerFactory.newController(), rpcTimeout, new RetryingTimeTracker(), PRIORITY_UNSET); - this.id = id; - this.location = location; - } - - /** - * Two responsibilities - * - if the call is already completed (by another replica) stops the retries. - * - set the location to the right region, depending on the replica. - */ - @Override - // TODO: Very like the super class implemenation. Can we shrink this down? - public void prepare(final boolean reload) throws IOException { - if (getRpcController().isCanceled()) return; - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - if (reload || location == null) { - RegionLocations rl = getRegionLocations(false, id, cConnection, tableName, get.getRow()); - location = id < rl.size() ? rl.getRegionLocation(id) : null; - } - - if (location == null || location.getServerName() == null) { - // With this exception, there will be a retry. The location can be null for a replica - // when the table is created or after a split. - throw new HBaseIOException("There is no location for replica id #" + id); - } - - setStubByServiceName(this.location.getServerName()); - } - - @Override - // TODO: Very like the super class implemenation. Can we shrink this down? - protected Result rpcCall() throws Exception { - if (getRpcController().isCanceled()) return null; - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - byte[] reg = location.getRegionInfo().getRegionName(); - ClientProtos.GetRequest request = RequestConverter.buildGetRequest(reg, get); - HBaseRpcController hrc = (HBaseRpcController)getRpcController(); - hrc.reset(); - hrc.setCallTimeout(rpcTimeout); - hrc.setPriority(tableName); - ClientProtos.GetResponse response = getStub().get(hrc, request); - if (response == null) { - return null; - } - return ProtobufUtil.toResult(response.getResult(), hrc.cellScanner()); - } - } - - /** - *

- * Algo: - * - we put the query into the execution pool. - * - after x ms, if we don't have a result, we add the queries for the secondary replicas - * - we take the first answer - * - when done, we cancel what's left. Cancelling means: - * - removing from the pool if the actual call was not started - * - interrupting the call if it has started - * Client side, we need to take into account - * - a call is not executed immediately after being put into the pool - * - a call is a thread. Let's not multiply the number of thread by the number of replicas. - * Server side, if we can cancel when it's still in the handler pool, it's much better, as a call - * can take some i/o. - *

- * Globally, the number of retries, timeout and so on still applies, but it's per replica, - * not global. We continue until all retries are done, or all timeouts are exceeded. - */ - public Result call(int operationTimeout) - throws DoNotRetryIOException, InterruptedIOException, RetriesExhaustedException { - boolean isTargetReplicaSpecified = (get.getReplicaId() >= 0); - - RegionLocations rl = null; - boolean skipPrimary = false; - try { - rl = getRegionLocations(true, - (isTargetReplicaSpecified ? get.getReplicaId() : RegionReplicaUtil.DEFAULT_REPLICA_ID), - cConnection, tableName, get.getRow()); - } catch (RetriesExhaustedException | DoNotRetryIOException e) { - // When there is no specific replica id specified. It just needs to load all replicas. - if (isTargetReplicaSpecified) { - throw e; - } else { - // We cannot get the primary replica location, it is possible that the region - // server hosting meta is down, it needs to proceed to try cached replicas. - rl = cConnection.getCachedLocation(tableName, get.getRow()); - if (rl == null) { - // No cached locations - throw e; - } - - // Primary replica location is not known, skip primary replica - skipPrimary = true; - } - } - - final ResultBoundedCompletionService cs = - new ResultBoundedCompletionService<>(this.rpcRetryingCallerFactory, pool, rl.size()); - int startIndex = 0; - int endIndex = rl.size(); - - if(isTargetReplicaSpecified) { - addCallsForReplica(cs, rl, get.getReplicaId(), get.getReplicaId()); - endIndex = 1; - } else { - if (!skipPrimary) { - addCallsForReplica(cs, rl, 0, 0); - try { - // wait for the timeout to see whether the primary responds back - Future f = cs.poll(timeBeforeReplicas, TimeUnit.MICROSECONDS); // Yes, microseconds - if (f != null) { - return f.get(); //great we got a response - } - if (cConnection.getConnectionMetrics() != null) { - cConnection.getConnectionMetrics().incrHedgedReadOps(); - } - } catch (ExecutionException e) { - // We ignore the ExecutionException and continue with the secondary replicas - if (LOG.isDebugEnabled()) { - LOG.debug("Primary replica returns " + e.getCause()); - } - - // Skip the result from the primary as we know that there is something wrong - startIndex = 1; - } catch (CancellationException e) { - throw new InterruptedIOException(); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } - } else { - // Since primary replica is skipped, the endIndex needs to be adjusted accordingly - endIndex --; - } - - // submit call for the all of the secondaries at once - addCallsForReplica(cs, rl, 1, rl.size() - 1); - } - try { - ResultBoundedCompletionService.QueueingFuture f = - cs.pollForFirstSuccessfullyCompletedTask(operationTimeout, TimeUnit.MILLISECONDS, startIndex, endIndex); - if (f == null) { - throw new RetriesExhaustedException("Timed out after " + operationTimeout + - "ms. Get is sent to replicas with startIndex: " + startIndex + - ", endIndex: " + endIndex + ", Locations: " + rl); - } - if (cConnection.getConnectionMetrics() != null && !isTargetReplicaSpecified && - !skipPrimary && f.getReplicaId() != RegionReplicaUtil.DEFAULT_REPLICA_ID) { - cConnection.getConnectionMetrics().incrHedgedReadWin(); - } - return f.get(); - } catch (ExecutionException e) { - throwEnrichedException(e, retries); - } catch (CancellationException e) { - throw new InterruptedIOException(); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } finally { - // We get there because we were interrupted or because one or more of the - // calls succeeded or failed. In all case, we stop all our tasks. - cs.cancelAll(); - } - - LOG.error("Imposible? Arrive at an unreachable line..."); // unreachable - return null; // unreachable - } - - /** - * Extract the real exception from the ExecutionException, and throws what makes more - * sense. - */ - static void throwEnrichedException(ExecutionException e, int retries) - throws RetriesExhaustedException, DoNotRetryIOException { - Throwable t = e.getCause(); - assert t != null; // That's what ExecutionException is about: holding an exception - - if (t instanceof RetriesExhaustedException) { - throw (RetriesExhaustedException) t; - } - - if (t instanceof DoNotRetryIOException) { - throw (DoNotRetryIOException) t; - } - - RetriesExhaustedException.ThrowableWithExtraContext qt = - new RetriesExhaustedException.ThrowableWithExtraContext(t, - EnvironmentEdgeManager.currentTime(), null); - - List exceptions = - Collections.singletonList(qt); - - throw new RetriesExhaustedException(retries, exceptions); - } - - /** - * Creates the calls and submit them - * - * @param cs - the completion service to use for submitting - * @param rl - the region locations - * @param min - the id of the first replica, inclusive - * @param max - the id of the last replica, inclusive. - */ - private void addCallsForReplica(ResultBoundedCompletionService cs, - RegionLocations rl, int min, int max) { - for (int id = min; id <= max; id++) { - HRegionLocation hrl = rl.getRegionLocation(id); - ReplicaRegionServerCallable callOnReplica = new ReplicaRegionServerCallable(id, hrl); - cs.submit(callOnReplica, operationTimeout, id); - } - } - - static RegionLocations getRegionLocations(boolean useCache, int replicaId, - ConnectionImplementation cConnection, TableName tableName, byte[] row) - throws RetriesExhaustedException, DoNotRetryIOException, InterruptedIOException { - RegionLocations rl; - try { - if (useCache) { - rl = cConnection.locateRegion(tableName, row, true, true, replicaId); - } else { - rl = cConnection.relocateRegion(tableName, row, replicaId); - } - } catch (DoNotRetryIOException | InterruptedIOException | RetriesExhaustedException e) { - throw e; - } catch (IOException e) { - throw new RetriesExhaustedException("Cannot get the location for replica" + replicaId - + " of region for " + Bytes.toStringBinary(row) + " in " + tableName, e); - } - if (rl == null) { - throw new RetriesExhaustedException("Cannot get the location for replica" + replicaId - + " of region for " + Bytes.toStringBinary(row) + " in " + tableName); - } - - return rl; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java deleted file mode 100644 index bf7135fab459..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ /dev/null @@ -1,424 +0,0 @@ -/** - * 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 static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCCallsMetrics; -import static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCRetriesMetrics; -import static org.apache.hadoop.hbase.client.ConnectionUtils.isRemote; -import static org.apache.hadoop.hbase.client.ConnectionUtils.updateResultsMetrics; -import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMetrics; - -import java.io.IOException; -import java.io.InterruptedIOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseIOException; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.NotServingRegionException; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.exceptions.ScannerResetException; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; - -/** - * Scanner operations such as create, next, etc. - * Used by {@link ResultScanner}s made by {@link Table}. Passed to a retrying caller such as - * {@link RpcRetryingCaller} so fails are retried. - */ -@InterfaceAudience.Private -public class ScannerCallable extends ClientServiceCallable { - public static final String LOG_SCANNER_LATENCY_CUTOFF - = "hbase.client.log.scanner.latency.cutoff"; - public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity"; - - // Keeping LOG public as it is being used in TestScannerHeartbeatMessages - public static final Logger LOG = LoggerFactory.getLogger(ScannerCallable.class); - protected long scannerId = -1L; - protected boolean instantiated = false; - protected boolean closed = false; - protected boolean renew = false; - protected final Scan scan; - private int caching = 1; - protected ScanMetrics scanMetrics; - private boolean logScannerActivity = false; - private int logCutOffLatency = 1000; - protected final int id; - - enum MoreResults { - YES, NO, UNKNOWN - } - - private MoreResults moreResultsInRegion; - private MoreResults moreResultsForScan; - - /** - * Saves whether or not the most recent response from the server was a heartbeat message. - * Heartbeat messages are identified by the flag {@link ScanResponse#getHeartbeatMessage()} - */ - protected boolean heartbeatMessage = false; - - protected Cursor cursor; - - // indicate if it is a remote server call - protected boolean isRegionServerRemote = true; - private long nextCallSeq = 0; - protected final RpcControllerFactory rpcControllerFactory; - - /** - * @param connection which connection - * @param tableName table callable is on - * @param scan the scan to execute - * @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable won't collect - * metrics - * @param rpcControllerFactory factory to use when creating - * {@link com.google.protobuf.RpcController} - */ - public ScannerCallable(ConnectionImplementation connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory) { - this(connection, tableName, scan, scanMetrics, rpcControllerFactory, 0); - } - - /** - * @param connection - * @param tableName - * @param scan - * @param scanMetrics - * @param id the replicaId - */ - public ScannerCallable(ConnectionImplementation connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) { - super(connection, tableName, scan.getStartRow(), rpcControllerFactory.newController(), - scan.getPriority()); - this.id = id; - this.scan = scan; - this.scanMetrics = scanMetrics; - Configuration conf = connection.getConfiguration(); - logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false); - logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000); - this.rpcControllerFactory = rpcControllerFactory; - } - - /** - * @param reload force reload of server location - * @throws IOException - */ - @Override - public void prepare(boolean reload) throws IOException { - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(!reload, - id, getConnection(), getTableName(), getRow()); - location = id < rl.size() ? rl.getRegionLocation(id) : null; - if (location == null || location.getServerName() == null) { - // With this exception, there will be a retry. The location can be null for a replica - // when the table is created or after a split. - throw new HBaseIOException("There is no location for replica id #" + id); - } - ServerName dest = location.getServerName(); - setStub(super.getConnection().getClient(dest)); - if (!instantiated || reload) { - checkIfRegionServerIsRemote(); - instantiated = true; - } - cursor = null; - // check how often we retry. - if (reload) { - incRPCRetriesMetrics(scanMetrics, isRegionServerRemote); - } - } - - /** - * compare the local machine hostname with region server's hostname to decide if hbase client - * connects to a remote region server - */ - protected void checkIfRegionServerIsRemote() { - isRegionServerRemote = isRemote(getLocation().getHostname()); - } - - private ScanResponse next() throws IOException { - // Reset the heartbeat flag prior to each RPC in case an exception is thrown by the server - setHeartbeatMessage(false); - incRPCCallsMetrics(scanMetrics, isRegionServerRemote); - ScanRequest request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq, - this.scanMetrics != null, renew, scan.getLimit()); - try { - ScanResponse response = getStub().scan(getRpcController(), request); - nextCallSeq++; - return response; - } catch (Exception e) { - IOException ioe = ProtobufUtil.handleRemoteException(e); - if (logScannerActivity) { - LOG.info( - "Got exception making request " + ProtobufUtil.toText(request) + " to " + getLocation(), - e); - } - if (logScannerActivity) { - if (ioe instanceof UnknownScannerException) { - try { - HRegionLocation location = - getConnection().relocateRegion(getTableName(), scan.getStartRow()); - LOG.info("Scanner=" + scannerId + " expired, current region location is " - + location.toString()); - } catch (Throwable t) { - LOG.info("Failed to relocate region", t); - } - } else if (ioe instanceof ScannerResetException) { - LOG.info("Scanner=" + scannerId + " has received an exception, and the server " - + "asked us to reset the scanner state.", - ioe); - } - } - // The below convertion of exceptions into DoNotRetryExceptions is a little strange. - // Why not just have these exceptions implment DNRIOE you ask? Well, usually we want - // ServerCallable#withRetries to just retry when it gets these exceptions. In here in - // a scan when doing a next in particular, we want to break out and get the scanner to - // reset itself up again. Throwing a DNRIOE is how we signal this to happen (its ugly, - // yeah and hard to follow and in need of a refactor). - if (ioe instanceof NotServingRegionException) { - // Throw a DNRE so that we break out of cycle of calling NSRE - // when what we need is to open scanner against new location. - // Attach NSRE to signal client that it needs to re-setup scanner. - if (this.scanMetrics != null) { - this.scanMetrics.countOfNSRE.incrementAndGet(); - } - throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); - } else if (ioe instanceof RegionServerStoppedException) { - // Throw a DNRE so that we break out of cycle of the retries and instead go and - // open scanner against new location. - throw new DoNotRetryIOException("Resetting the scanner -- see exception cause", ioe); - } else { - // The outer layers will retry - throw ioe; - } - } - } - - private void setAlreadyClosed() { - this.scannerId = -1L; - this.closed = true; - } - - @Override - protected Result[] rpcCall() throws Exception { - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - if (closed) { - close(); - return null; - } - ScanResponse response; - if (this.scannerId == -1L) { - response = openScanner(); - } else { - response = next(); - } - long timestamp = System.currentTimeMillis(); - boolean isHeartBeat = response.hasHeartbeatMessage() && response.getHeartbeatMessage(); - setHeartbeatMessage(isHeartBeat); - if (isHeartBeat && scan.isNeedCursorResult() && response.hasCursor()) { - cursor = ProtobufUtil.toCursor(response.getCursor()); - } - Result[] rrs = ResponseConverter.getResults(getRpcControllerCellScanner(), response); - if (logScannerActivity) { - long now = System.currentTimeMillis(); - if (now - timestamp > logCutOffLatency) { - int rows = rrs == null ? 0 : rrs.length; - LOG.info("Took " + (now - timestamp) + "ms to fetch " + rows + " rows from scanner=" - + scannerId); - } - } - updateServerSideMetrics(scanMetrics, response); - // moreResults is only used for the case where a filter exhausts all elements - if (response.hasMoreResults()) { - if (response.getMoreResults()) { - setMoreResultsForScan(MoreResults.YES); - } else { - setMoreResultsForScan(MoreResults.NO); - setAlreadyClosed(); - } - } else { - setMoreResultsForScan(MoreResults.UNKNOWN); - } - if (response.hasMoreResultsInRegion()) { - if (response.getMoreResultsInRegion()) { - setMoreResultsInRegion(MoreResults.YES); - } else { - setMoreResultsInRegion(MoreResults.NO); - setAlreadyClosed(); - } - } else { - setMoreResultsInRegion(MoreResults.UNKNOWN); - } - updateResultsMetrics(scanMetrics, rrs, isRegionServerRemote); - return rrs; - } - - /** - * @return true when the most recent RPC response indicated that the response was a heartbeat - * message. Heartbeat messages are sent back from the server when the processing of the - * scan request exceeds a certain time threshold. Heartbeats allow the server to avoid - * timeouts during long running scan operations. - */ - boolean isHeartbeatMessage() { - return heartbeatMessage; - } - - public Cursor getCursor() { - return cursor; - } - - private void setHeartbeatMessage(boolean heartbeatMessage) { - this.heartbeatMessage = heartbeatMessage; - } - - private void close() { - if (this.scannerId == -1L) { - return; - } - try { - incRPCCallsMetrics(scanMetrics, isRegionServerRemote); - ScanRequest request = - RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null); - try { - getStub().scan(getRpcController(), request); - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } catch (IOException e) { - TableName table = getTableName(); - String tableDetails = (table == null) ? "" : (" on table: " + table.getNameAsString()); - LOG.warn("Ignore, probably already closed. Current scan: " + getScan().toString() - + tableDetails, e); - } - this.scannerId = -1L; - } - - private ScanResponse openScanner() throws IOException { - incRPCCallsMetrics(scanMetrics, isRegionServerRemote); - ScanRequest request = RequestConverter.buildScanRequest( - getLocation().getRegionInfo().getRegionName(), this.scan, this.caching, false); - try { - ScanResponse response = getStub().scan(getRpcController(), request); - long id = response.getScannerId(); - if (logScannerActivity) { - LOG.info("Open scanner=" + id + " for scan=" + scan.toString() - + " on region " + getLocation().toString()); - } - if (response.hasMvccReadPoint()) { - this.scan.setMvccReadPoint(response.getMvccReadPoint()); - } - this.scannerId = id; - return response; - } catch (Exception e) { - throw ProtobufUtil.handleRemoteException(e); - } - } - - protected Scan getScan() { - return scan; - } - - /** - * Call this when the next invocation of call should close the scanner - */ - public void setClose() { - this.closed = true; - } - - /** - * Indicate whether we make a call only to renew the lease, but without affected the scanner in - * any other way. - * @param val true if only the lease should be renewed - */ - public void setRenew(boolean val) { - this.renew = val; - } - - /** - * @return the HRegionInfo for the current region - */ - @Override - public HRegionInfo getHRegionInfo() { - if (!instantiated) { - return null; - } - return getLocation().getRegionInfo(); - } - - /** - * Get the number of rows that will be fetched on next - * @return the number of rows for caching - */ - public int getCaching() { - return caching; - } - - /** - * Set the number of rows that will be fetched on next - * @param caching the number of rows for caching - */ - public void setCaching(int caching) { - this.caching = caching; - } - - public ScannerCallable getScannerCallableForReplica(int id) { - ScannerCallable s = new ScannerCallable(this.getConnection(), getTableName(), - this.getScan(), this.scanMetrics, this.rpcControllerFactory, id); - s.setCaching(this.caching); - return s; - } - - /** - * Should the client attempt to fetch more results from this region - */ - MoreResults moreResultsInRegion() { - return moreResultsInRegion; - } - - void setMoreResultsInRegion(MoreResults moreResults) { - this.moreResultsInRegion = moreResults; - } - - /** - * Should the client attempt to fetch more results for the whole scan. - */ - MoreResults moreResultsForScan() { - return moreResultsForScan; - } - - void setMoreResultsForScan(MoreResults moreResults) { - this.moreResultsForScan = moreResults; - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java deleted file mode 100644 index db956ce81cf8..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java +++ /dev/null @@ -1,444 +0,0 @@ -/** - * 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.HashSet; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - -/** - * This class has the logic for handling scanners for regions with and without replicas. - * 1. A scan is attempted on the default (primary) region - * 2. The scanner sends all the RPCs to the default region until it is done, or, there - * is a timeout on the default (a timeout of zero is disallowed). - * 3. If there is a timeout in (2) above, scanner(s) is opened on the non-default replica(s) - * 4. The results from the first successful scanner are taken, and it is stored which server - * returned the results. - * 5. The next RPCs are done on the above stored server until it is done or there is a timeout, - * in which case, the other replicas are queried (as in (3) above). - * - */ -@InterfaceAudience.Private -class ScannerCallableWithReplicas implements RetryingCallable { - private static final Logger LOG = LoggerFactory.getLogger(ScannerCallableWithReplicas.class); - volatile ScannerCallable currentScannerCallable; - AtomicBoolean replicaSwitched = new AtomicBoolean(false); - final ConnectionImplementation cConnection; - protected final ExecutorService pool; - protected final int timeBeforeReplicas; - private final Scan scan; - private final int retries; - private Result lastResult; - private final RpcRetryingCaller caller; - private final TableName tableName; - private Configuration conf; - private int scannerTimeout; - private Set outstandingCallables = new HashSet<>(); - private boolean someRPCcancelled = false; //required for testing purposes only - private int regionReplication = 0; - - public ScannerCallableWithReplicas(TableName tableName, ConnectionImplementation cConnection, - ScannerCallable baseCallable, ExecutorService pool, int timeBeforeReplicas, Scan scan, - int retries, int scannerTimeout, int caching, Configuration conf, - RpcRetryingCaller caller) { - this.currentScannerCallable = baseCallable; - this.cConnection = cConnection; - this.pool = pool; - if (timeBeforeReplicas < 0) { - throw new IllegalArgumentException("Invalid value of operation timeout on the primary"); - } - this.timeBeforeReplicas = timeBeforeReplicas; - this.scan = scan; - this.retries = retries; - this.tableName = tableName; - this.conf = conf; - this.scannerTimeout = scannerTimeout; - this.caller = caller; - } - - public void setClose() { - currentScannerCallable.setClose(); - } - - public void setRenew(boolean val) { - currentScannerCallable.setRenew(val); - } - - public void setCaching(int caching) { - currentScannerCallable.setCaching(caching); - } - - public int getCaching() { - return currentScannerCallable.getCaching(); - } - - public HRegionInfo getHRegionInfo() { - return currentScannerCallable.getHRegionInfo(); - } - - public MoreResults moreResultsInRegion() { - return currentScannerCallable.moreResultsInRegion(); - } - - public MoreResults moreResultsForScan() { - return currentScannerCallable.moreResultsForScan(); - } - - @Override - public Result [] call(int timeout) throws IOException { - // If the active replica callable was closed somewhere, invoke the RPC to - // really close it. In the case of regular scanners, this applies. We make couple - // of RPCs to a RegionServer, and when that region is exhausted, we set - // the closed flag. Then an RPC is required to actually close the scanner. - if (currentScannerCallable != null && currentScannerCallable.closed) { - // For closing we target that exact scanner (and not do replica fallback like in - // the case of normal reads) - if (LOG.isTraceEnabled()) { - LOG.trace("Closing scanner id=" + currentScannerCallable.scannerId); - } - Result[] r = currentScannerCallable.call(timeout); - currentScannerCallable = null; - return r; - } - // We need to do the following: - //1. When a scan goes out to a certain replica (default or not), we need to - // continue to hit that until there is a failure. So store the last successfully invoked - // replica - //2. We should close the "losing" scanners (scanners other than the ones we hear back - // from first) - // - // Since RegionReplication is a table attribute, it wont change as long as table is enabled, - // it just needs to be set once. - - if (regionReplication <= 0) { - RegionLocations rl = null; - try { - rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(true, - RegionReplicaUtil.DEFAULT_REPLICA_ID, cConnection, tableName, - currentScannerCallable.getRow()); - } catch (RetriesExhaustedException | DoNotRetryIOException e) { - // We cannot get the primary replica region location, it is possible that the region server - // hosting meta table is down, it needs to proceed to try cached replicas directly. - rl = cConnection.getCachedLocation(tableName, currentScannerCallable.getRow()); - if (rl == null) { - throw e; - } - } - regionReplication = rl.size(); - } - // allocate a boundedcompletion pool of some multiple of number of replicas. - // We want to accomodate some RPCs for redundant replica scans (but are still in progress) - ResultBoundedCompletionService> cs = - new ResultBoundedCompletionService<>( - RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf), pool, - regionReplication * 5); - - AtomicBoolean done = new AtomicBoolean(false); - replicaSwitched.set(false); - // submit call for the primary replica. - addCallsForCurrentReplica(cs); - int startIndex = 0; - - try { - // wait for the timeout to see whether the primary responds back - Future> f = cs.poll(timeBeforeReplicas, - TimeUnit.MICROSECONDS); // Yes, microseconds - if (f != null) { - // After poll, if f is not null, there must be a completed task - Pair r = f.get(); - if (r != null && r.getSecond() != null) { - updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); - } - return r == null ? null : r.getFirst(); //great we got a response - } - } catch (ExecutionException e) { - // We ignore the ExecutionException and continue with the replicas - if (LOG.isDebugEnabled()) { - LOG.debug("Scan with primary region returns " + e.getCause()); - } - - // If rl's size is 1 or scan's consitency is strong, it needs to throw - // out the exception from the primary replica - if ((regionReplication == 1) || (scan.getConsistency() == Consistency.STRONG)) { - // Rethrow the first exception - RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries); - } - - startIndex = 1; - } catch (CancellationException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } - - // submit call for the all of the secondaries at once - int endIndex = regionReplication; - if (scan.getConsistency() == Consistency.STRONG) { - // When scan's consistency is strong, do not send to the secondaries - endIndex = 1; - } else { - // TODO: this may be an overkill for large region replication - addCallsForOtherReplicas(cs, 0, regionReplication - 1); - } - - try { - Future> f = cs.pollForFirstSuccessfullyCompletedTask(timeout, - TimeUnit.MILLISECONDS, startIndex, endIndex); - - if (f == null) { - throw new IOException("Failed to get result within timeout, timeout=" + timeout + "ms"); - } - Pair r = f.get(); - - if (r != null && r.getSecond() != null) { - updateCurrentlyServingReplica(r.getSecond(), r.getFirst(), done, pool); - } - return r == null ? null : r.getFirst(); // great we got an answer - - } catch (ExecutionException e) { - RpcRetryingCallerWithReadReplicas.throwEnrichedException(e, retries); - } catch (CancellationException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } finally { - // We get there because we were interrupted or because one or more of the - // calls succeeded or failed. In all case, we stop all our tasks. - cs.cancelAll(); - } - LOG.error("Imposible? Arrive at an unreachable line..."); // unreachable - throw new IOException("Imposible? Arrive at an unreachable line..."); - } - - private void updateCurrentlyServingReplica(ScannerCallable scanner, Result[] result, - AtomicBoolean done, ExecutorService pool) { - if (done.compareAndSet(false, true)) { - if (currentScannerCallable != scanner) replicaSwitched.set(true); - currentScannerCallable = scanner; - // store where to start the replica scanner from if we need to. - if (result != null && result.length != 0) this.lastResult = result[result.length - 1]; - if (LOG.isTraceEnabled()) { - LOG.trace("Setting current scanner as id=" + currentScannerCallable.scannerId + - " associated with replica=" + currentScannerCallable.getHRegionInfo().getReplicaId()); - } - // close all outstanding replica scanners but the one we heard back from - outstandingCallables.remove(scanner); - for (ScannerCallable s : outstandingCallables) { - if (LOG.isTraceEnabled()) { - LOG.trace("Closing scanner id=" + s.scannerId + - ", replica=" + s.getHRegionInfo().getRegionId() + - " because slow and replica=" + - this.currentScannerCallable.getHRegionInfo().getReplicaId() + " succeeded"); - } - // Submit the "close" to the pool since this might take time, and we don't - // want to wait for the "close" to happen yet. The "wait" will happen when - // the table is closed (when the awaitTermination of the underlying pool is called) - s.setClose(); - final RetryingRPC r = new RetryingRPC(s); - pool.submit(new Callable(){ - @Override - public Void call() throws Exception { - r.call(scannerTimeout); - return null; - } - }); - } - // now clear outstandingCallables since we scheduled a close for all the contained scanners - outstandingCallables.clear(); - } - } - - /** - * When a scanner switches in the middle of scanning (the 'next' call fails - * for example), the upper layer {@link ClientScanner} needs to know - */ - public boolean switchedToADifferentReplica() { - return replicaSwitched.get(); - } - - /** - * @return true when the most recent RPC response indicated that the response was a heartbeat - * message. Heartbeat messages are sent back from the server when the processing of the - * scan request exceeds a certain time threshold. Heartbeats allow the server to avoid - * timeouts during long running scan operations. - */ - public boolean isHeartbeatMessage() { - return currentScannerCallable != null && currentScannerCallable.isHeartbeatMessage(); - } - - public Cursor getCursor() { - return currentScannerCallable != null ? currentScannerCallable.getCursor() : null; - } - - private void addCallsForCurrentReplica( - ResultBoundedCompletionService> cs) { - RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable); - outstandingCallables.add(currentScannerCallable); - cs.submit(retryingOnReplica, scannerTimeout, currentScannerCallable.id); - } - - private void addCallsForOtherReplicas( - ResultBoundedCompletionService> cs, int min, int max) { - - for (int id = min; id <= max; id++) { - if (currentScannerCallable.id == id) { - continue; //this was already scheduled earlier - } - ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id); - setStartRowForReplicaCallable(s); - outstandingCallables.add(s); - RetryingRPC retryingOnReplica = new RetryingRPC(s); - cs.submit(retryingOnReplica, scannerTimeout, id); - } - } - - /** - * Set the start row for the replica callable based on the state of the last result received. - * @param callable The callable to set the start row on - */ - private void setStartRowForReplicaCallable(ScannerCallable callable) { - if (this.lastResult == null || callable == null) { - return; - } - // 1. The last result was a partial result which means we have not received all of the cells - // for this row. Thus, use the last result's row as the start row. If a replica switch - // occurs, the scanner will ensure that any accumulated partial results are cleared, - // and the scan can resume from this row. - // 2. The last result was not a partial result which means it contained all of the cells for - // that row (we no longer need any information from it). Set the start row to the next - // closest row that could be seen. - callable.getScan().withStartRow(this.lastResult.getRow(), this.lastResult.mayHaveMoreCellsInRow()); - } - - @VisibleForTesting - boolean isAnyRPCcancelled() { - return someRPCcancelled; - } - - class RetryingRPC implements RetryingCallable>, Cancellable { - final ScannerCallable callable; - RpcRetryingCaller caller; - private volatile boolean cancelled = false; - - RetryingRPC(ScannerCallable callable) { - this.callable = callable; - // For the Consistency.STRONG (default case), we reuse the caller - // to keep compatibility with what is done in the past - // For the Consistency.TIMELINE case, we can't reuse the caller - // since we could be making parallel RPCs (caller.callWithRetries is synchronized - // and we can't invoke it multiple times at the same time) - this.caller = ScannerCallableWithReplicas.this.caller; - if (scan.getConsistency() == Consistency.TIMELINE) { - this.caller = RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf) - .newCaller(); - } - } - - @Override - public Pair call(int callTimeout) throws IOException { - // since the retries is done within the ResultBoundedCompletionService, - // we don't invoke callWithRetries here - if (cancelled) { - return null; - } - Result[] res = this.caller.callWithoutRetries(this.callable, callTimeout); - return new Pair<>(res, this.callable); - } - - @Override - public void prepare(boolean reload) throws IOException { - if (cancelled) return; - - if (Thread.interrupted()) { - throw new InterruptedIOException(); - } - - callable.prepare(reload); - } - - @Override - public void throwable(Throwable t, boolean retrying) { - callable.throwable(t, retrying); - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return callable.getExceptionMessageAdditionalDetail(); - } - - @Override - public long sleep(long pause, int tries) { - return callable.sleep(pause, tries); - } - - @Override - public void cancel() { - cancelled = true; - caller.cancel(); - if (callable.getRpcController() != null) { - callable.getRpcController().startCancel(); - } - someRPCcancelled = true; - } - - @Override - public boolean isCancelled() { - return cancelled; - } - } - - @Override - public void prepare(boolean reload) throws IOException { - } - - @Override - public void throwable(Throwable t, boolean retrying) { - currentScannerCallable.throwable(t, retrying); - } - - @Override - public String getExceptionMessageAdditionalDetail() { - return currentScannerCallable.getExceptionMessageAdditionalDetail(); - } - - @Override - public long sleep(long pause, int tries) { - return currentScannerCallable.sleep(pause, tries); - } -} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java deleted file mode 100644 index 892761549163..000000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java +++ /dev/null @@ -1,707 +0,0 @@ -/** - * 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.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasUserPermissionsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClearDeadServersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsRpcThrottleEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDecommissionedRegionServersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RecommissionRegionServerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchExceedThrottleQuotaResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchRpcThrottleResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; - -/** - * A short-circuit connection that can bypass the RPC layer (serialization, deserialization, - * networking, etc..) when talking to a local master - */ -@InterfaceAudience.Private -public class ShortCircuitMasterConnection implements MasterKeepAliveConnection { - - private final MasterService.BlockingInterface stub; - - public ShortCircuitMasterConnection(MasterService.BlockingInterface stub) { - this.stub = stub; - } - - @Override - public UnassignRegionResponse unassignRegion(RpcController controller, - UnassignRegionRequest request) throws ServiceException { - return stub.unassignRegion(controller, request); - } - - @Override - public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request) - throws ServiceException { - return stub.truncateTable(controller, request); - } - - @Override - public StopMasterResponse stopMaster(RpcController controller, StopMasterRequest request) - throws ServiceException { - return stub.stopMaster(controller, request); - } - - @Override - public SnapshotResponse snapshot(RpcController controller, SnapshotRequest request) - throws ServiceException { - return stub.snapshot(controller, request); - } - - @Override - public ShutdownResponse shutdown(RpcController controller, ShutdownRequest request) - throws ServiceException { - return stub.shutdown(controller, request); - } - - @Override - public SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled(RpcController controller, - SetSplitOrMergeEnabledRequest request) throws ServiceException { - return stub.setSplitOrMergeEnabled(controller, request); - } - - @Override - public SetQuotaResponse setQuota(RpcController controller, SetQuotaRequest request) - throws ServiceException { - return stub.setQuota(controller, request); - } - - @Override - public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller, - SetNormalizerRunningRequest request) throws ServiceException { - return stub.setNormalizerRunning(controller, request); - } - - @Override - public SetBalancerRunningResponse setBalancerRunning(RpcController controller, - SetBalancerRunningRequest request) throws ServiceException { - return stub.setBalancerRunning(controller, request); - } - - @Override - public RunCatalogScanResponse runCatalogScan(RpcController controller, - RunCatalogScanRequest request) throws ServiceException { - return stub.runCatalogScan(controller, request); - } - - @Override - public RestoreSnapshotResponse restoreSnapshot(RpcController controller, - RestoreSnapshotRequest request) throws ServiceException { - return stub.restoreSnapshot(controller, request); - } - - @Override - public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller, - RemoveReplicationPeerRequest request) throws ServiceException { - return stub.removeReplicationPeer(controller, request); - } - - @Override - public RecommissionRegionServerResponse recommissionRegionServer(RpcController controller, - RecommissionRegionServerRequest request) throws ServiceException { - return stub.recommissionRegionServer(controller, request); - } - - @Override - public OfflineRegionResponse offlineRegion(RpcController controller, OfflineRegionRequest request) - throws ServiceException { - return stub.offlineRegion(controller, request); - } - - @Override - public NormalizeResponse normalize(RpcController controller, NormalizeRequest request) - throws ServiceException { - return stub.normalize(controller, request); - } - - @Override - public MoveRegionResponse moveRegion(RpcController controller, MoveRegionRequest request) - throws ServiceException { - return stub.moveRegion(controller, request); - } - - @Override - public ModifyTableResponse modifyTable(RpcController controller, ModifyTableRequest request) - throws ServiceException { - return stub.modifyTable(controller, request); - } - - @Override - public ModifyNamespaceResponse modifyNamespace(RpcController controller, - ModifyNamespaceRequest request) throws ServiceException { - return stub.modifyNamespace(controller, request); - } - - @Override - public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest request) - throws ServiceException { - return stub.modifyColumn(controller, request); - } - - @Override - public MergeTableRegionsResponse mergeTableRegions(RpcController controller, - MergeTableRegionsRequest request) throws ServiceException { - return stub.mergeTableRegions(controller, request); - } - - @Override - public ListTableNamesByNamespaceResponse listTableNamesByNamespace(RpcController controller, - ListTableNamesByNamespaceRequest request) throws ServiceException { - return stub.listTableNamesByNamespace(controller, request); - } - - @Override - public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace( - RpcController controller, ListTableDescriptorsByNamespaceRequest request) - throws ServiceException { - return stub.listTableDescriptorsByNamespace(controller, request); - } - - @Override - public GetProceduresResponse getProcedures(RpcController controller, - GetProceduresRequest request) throws ServiceException { - return stub.getProcedures(controller, request); - } - - @Override - public GetLocksResponse getLocks(RpcController controller, - GetLocksRequest request) throws ServiceException { - return stub.getLocks(controller, request); - } - - @Override - public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller, - ListNamespaceDescriptorsRequest request) throws ServiceException { - return stub.listNamespaceDescriptors(controller, request); - } - - @Override - public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(RpcController controller, - ListDecommissionedRegionServersRequest request) throws ServiceException { - return stub.listDecommissionedRegionServers(controller, request); - } - - @Override - public IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled(RpcController controller, - IsSplitOrMergeEnabledRequest request) throws ServiceException { - return stub.isSplitOrMergeEnabled(controller, request); - } - - @Override - public IsSnapshotDoneResponse isSnapshotDone(RpcController controller, - IsSnapshotDoneRequest request) throws ServiceException { - return stub.isSnapshotDone(controller, request); - } - - @Override - public IsProcedureDoneResponse isProcedureDone(RpcController controller, - IsProcedureDoneRequest request) throws ServiceException { - return stub.isProcedureDone(controller, request); - } - - @Override - public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController controller, - IsNormalizerEnabledRequest request) throws ServiceException { - return stub.isNormalizerEnabled(controller, request); - } - - @Override - public IsMasterRunningResponse isMasterRunning(RpcController controller, - IsMasterRunningRequest request) throws ServiceException { - return stub.isMasterRunning(controller, request); - } - - @Override - public IsInMaintenanceModeResponse isMasterInMaintenanceMode(RpcController controller, - IsInMaintenanceModeRequest request) throws ServiceException { - return stub.isMasterInMaintenanceMode(controller, request); - } - - @Override - public IsCatalogJanitorEnabledResponse isCatalogJanitorEnabled(RpcController controller, - IsCatalogJanitorEnabledRequest request) throws ServiceException { - return stub.isCatalogJanitorEnabled(controller, request); - } - - @Override - public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, - IsBalancerEnabledRequest request) throws ServiceException { - return stub.isBalancerEnabled(controller, request); - } - - @Override - public GetTableStateResponse getTableState(RpcController controller, GetTableStateRequest request) - throws ServiceException { - return stub.getTableState(controller, request); - } - - @Override - public GetTableNamesResponse getTableNames(RpcController controller, GetTableNamesRequest request) - throws ServiceException { - return stub.getTableNames(controller, request); - } - - @Override - public GetTableDescriptorsResponse getTableDescriptors(RpcController controller, - GetTableDescriptorsRequest request) throws ServiceException { - return stub.getTableDescriptors(controller, request); - } - - @Override - public SecurityCapabilitiesResponse getSecurityCapabilities(RpcController controller, - SecurityCapabilitiesRequest request) throws ServiceException { - return stub.getSecurityCapabilities(controller, request); - } - - @Override - public GetSchemaAlterStatusResponse getSchemaAlterStatus(RpcController controller, - GetSchemaAlterStatusRequest request) throws ServiceException { - return stub.getSchemaAlterStatus(controller, request); - } - - @Override - public GetProcedureResultResponse getProcedureResult(RpcController controller, - GetProcedureResultRequest request) throws ServiceException { - return stub.getProcedureResult(controller, request); - } - - @Override - public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller, - GetNamespaceDescriptorRequest request) throws ServiceException { - return stub.getNamespaceDescriptor(controller, request); - } - - @Override - public MajorCompactionTimestampResponse getLastMajorCompactionTimestampForRegion( - RpcController controller, MajorCompactionTimestampForRegionRequest request) - throws ServiceException { - return stub.getLastMajorCompactionTimestampForRegion(controller, request); - } - - @Override - public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(RpcController controller, - MajorCompactionTimestampRequest request) throws ServiceException { - return stub.getLastMajorCompactionTimestamp(controller, request); - } - - @Override - public GetCompletedSnapshotsResponse getCompletedSnapshots(RpcController controller, - GetCompletedSnapshotsRequest request) throws ServiceException { - return stub.getCompletedSnapshots(controller, request); - } - - @Override - public GetClusterStatusResponse getClusterStatus(RpcController controller, - GetClusterStatusRequest request) throws ServiceException { - return stub.getClusterStatus(controller, request); - } - - @Override - public ExecProcedureResponse execProcedureWithRet(RpcController controller, - ExecProcedureRequest request) throws ServiceException { - return stub.execProcedureWithRet(controller, request); - } - - @Override - public ExecProcedureResponse execProcedure(RpcController controller, ExecProcedureRequest request) - throws ServiceException { - return stub.execProcedure(controller, request); - } - - @Override - public CoprocessorServiceResponse execMasterService(RpcController controller, - CoprocessorServiceRequest request) throws ServiceException { - return stub.execMasterService(controller, request); - } - - @Override - public EnableTableResponse enableTable(RpcController controller, EnableTableRequest request) - throws ServiceException { - return stub.enableTable(controller, request); - } - - @Override - public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller, - EnableReplicationPeerRequest request) throws ServiceException { - return stub.enableReplicationPeer(controller, request); - } - - @Override - public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController controller, - EnableCatalogJanitorRequest request) throws ServiceException { - return stub.enableCatalogJanitor(controller, request); - } - - @Override - public DecommissionRegionServersResponse decommissionRegionServers(RpcController controller, - DecommissionRegionServersRequest request) throws ServiceException { - return stub.decommissionRegionServers(controller, request); - } - - @Override - public DisableTableResponse disableTable(RpcController controller, DisableTableRequest request) - throws ServiceException { - return stub.disableTable(controller, request); - } - - @Override - public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller, - DisableReplicationPeerRequest request) throws ServiceException { - return stub.disableReplicationPeer(controller, request); - } - - @Override - public DeleteTableResponse deleteTable(RpcController controller, DeleteTableRequest request) - throws ServiceException { - return stub.deleteTable(controller, request); - } - - @Override - public DeleteSnapshotResponse deleteSnapshot(RpcController controller, - DeleteSnapshotRequest request) throws ServiceException { - return stub.deleteSnapshot(controller, request); - } - - @Override - public DeleteNamespaceResponse deleteNamespace(RpcController controller, - DeleteNamespaceRequest request) throws ServiceException { - return stub.deleteNamespace(controller, request); - } - - @Override - public DeleteColumnResponse deleteColumn(RpcController controller, DeleteColumnRequest request) - throws ServiceException { - return stub.deleteColumn(controller, request); - } - - @Override - public CreateTableResponse createTable(RpcController controller, CreateTableRequest request) - throws ServiceException { - return stub.createTable(controller, request); - } - - @Override - public CreateNamespaceResponse createNamespace(RpcController controller, - CreateNamespaceRequest request) throws ServiceException { - return stub.createNamespace(controller, request); - } - - @Override - public BalanceResponse balance(RpcController controller, BalanceRequest request) - throws ServiceException { - return stub.balance(controller, request); - } - - @Override - public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest request) - throws ServiceException { - return stub.assignRegion(controller, request); - } - - @Override - public AddReplicationPeerResponse addReplicationPeer(RpcController controller, - AddReplicationPeerRequest request) throws ServiceException { - return stub.addReplicationPeer(controller, request); - } - - @Override - public AddColumnResponse addColumn(RpcController controller, AddColumnRequest request) - throws ServiceException { - return stub.addColumn(controller, request); - } - - @Override - public AbortProcedureResponse abortProcedure(RpcController controller, - AbortProcedureRequest request) throws ServiceException { - return stub.abortProcedure(controller, request); - } - - @Override - public void close() { - // nothing to do here - } - - @Override - public RunCleanerChoreResponse runCleanerChore(RpcController controller, - RunCleanerChoreRequest request) throws ServiceException { - return stub.runCleanerChore(controller, request); - } - - @Override - public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController controller, - SetCleanerChoreRunningRequest request) throws ServiceException { - return stub.setCleanerChoreRunning(controller, request); - } - - @Override - public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(RpcController controller, - IsCleanerChoreEnabledRequest request) throws ServiceException { - return stub.isCleanerChoreEnabled(controller, request); - } - - @Override - public GetReplicationPeerConfigResponse getReplicationPeerConfig(RpcController controller, - GetReplicationPeerConfigRequest request) throws ServiceException { - return stub.getReplicationPeerConfig(controller, request); - } - - @Override - public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller, - UpdateReplicationPeerConfigRequest request) throws ServiceException { - return stub.updateReplicationPeerConfig(controller, request); - } - - @Override - public ListReplicationPeersResponse listReplicationPeers(RpcController controller, - ListReplicationPeersRequest request) throws ServiceException { - return stub.listReplicationPeers(controller, request); - } - - @Override - public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(RpcController controller, - GetSpaceQuotaRegionSizesRequest request) throws ServiceException { - return stub.getSpaceQuotaRegionSizes(controller, request); - } - - @Override - public GetQuotaStatesResponse getQuotaStates(RpcController controller, - GetQuotaStatesRequest request) throws ServiceException { - return stub.getQuotaStates(controller, request); - } - - @Override - public ClearDeadServersResponse clearDeadServers(RpcController controller, - ClearDeadServersRequest request) throws ServiceException { - return stub.clearDeadServers(controller, request); - } - - @Override - public SplitTableRegionResponse splitRegion(RpcController controller, SplitTableRegionRequest request) - throws ServiceException { - return stub.splitRegion(controller, request); - } - - @Override - public TransitReplicationPeerSyncReplicationStateResponse - transitReplicationPeerSyncReplicationState(RpcController controller, - TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException { - return stub.transitReplicationPeerSyncReplicationState(controller, request); - } - - @Override - public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller, - SwitchRpcThrottleRequest request) throws ServiceException { - return stub.switchRpcThrottle(controller, request); - } - - @Override - public IsRpcThrottleEnabledResponse isRpcThrottleEnabled(RpcController controller, - IsRpcThrottleEnabledRequest request) throws ServiceException { - return stub.isRpcThrottleEnabled(controller, request); - } - - @Override - public SwitchExceedThrottleQuotaResponse switchExceedThrottleQuota(RpcController controller, - SwitchExceedThrottleQuotaRequest request) throws ServiceException { - return stub.switchExceedThrottleQuota(controller, request); - } - - @Override - public GrantResponse grant(RpcController controller, GrantRequest request) - throws ServiceException { - return stub.grant(controller, request); - } - - @Override - public RevokeResponse revoke(RpcController controller, RevokeRequest request) - throws ServiceException { - return stub.revoke(controller, request); - } - - @Override - public GetUserPermissionsResponse getUserPermissions(RpcController controller, - GetUserPermissionsRequest request) throws ServiceException { - return stub.getUserPermissions(controller, request); - } - - @Override - public HasUserPermissionsResponse hasUserPermissions(RpcController controller, - HasUserPermissionsRequest request) throws ServiceException { - return stub.hasUserPermissions(controller, request); - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java deleted file mode 100644 index fca6fe1a281a..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ /dev/null @@ -1,1881 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CallQueueTooBigException; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseClassTestRule; -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.client.AsyncProcessTask.ListRowAccess; -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.client.coprocessor.Batch; -import org.apache.hadoop.hbase.exceptions.RegionOpeningException; -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; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category({ClientTests.class, MediumTests.class}) -public class TestAsyncProcess { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestAsyncProcess.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestAsyncProcess.class); - private static final TableName DUMMY_TABLE = - TableName.valueOf("DUMMY_TABLE"); - private static final byte[] DUMMY_BYTES_1 = Bytes.toBytes("DUMMY_BYTES_1"); - private static final byte[] DUMMY_BYTES_2 = Bytes.toBytes("DUMMY_BYTES_2"); - private static final byte[] DUMMY_BYTES_3 = Bytes.toBytes("DUMMY_BYTES_3"); - private static final byte[] FAILS = Bytes.toBytes("FAILS"); - private Configuration CONF; - private ConnectionConfiguration CONNECTION_CONFIG; - 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); - - // Replica stuff - private static final RegionInfo hri1r1 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 1); - private static final RegionInfo hri1r2 = RegionReplicaUtil.getRegionInfoForReplica(hri1, 2); - private static final RegionInfo 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 final RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2), - new HRegionLocation(hri2r1, sn3)); - 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 final int NB_RETRIES = 3; - - private int RPC_TIMEOUT; - private int OPERATION_TIMEOUT; - - @Before - public void beforeEach() { - this.CONF = new Configuration(); - CONF.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, NB_RETRIES); - this.CONNECTION_CONFIG = new ConnectionConfiguration(CONF); - this.RPC_TIMEOUT = CONF.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); - this.OPERATION_TIMEOUT = CONF.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); - } - - static class CountingThreadFactory implements ThreadFactory { - final AtomicInteger nbThreads; - ThreadFactory realFactory = Threads.newDaemonThreadFactory("test-TestAsyncProcess"); - @Override - public Thread newThread(Runnable r) { - nbThreads.incrementAndGet(); - return realFactory.newThread(r); - } - - CountingThreadFactory(AtomicInteger nbThreads){ - this.nbThreads = nbThreads; - } - } - - static class MyAsyncProcess extends AsyncProcess { - final AtomicInteger nbMultiResponse = new AtomicInteger(); - final AtomicInteger nbActions = new AtomicInteger(); - public List allReqs = new ArrayList<>(); - public AtomicInteger callsCt = new AtomicInteger(); - private Configuration conf; - - private long previousTimeout = -1; - final ExecutorService service; - @Override - 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<>( - wrap, actions, nonceGroup, this); - allReqs.add(r); - return r; - } - - public MyAsyncProcess(ConnectionImplementation hc, Configuration conf) { - super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf)); - service = Executors.newFixedThreadPool(5); - this.conf = conf; - } - - public MyAsyncProcess(ConnectionImplementation hc, Configuration conf, - AtomicInteger nbThreads) { - super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf)); - service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, new SynchronousQueue<>(), - new CountingThreadFactory(nbThreads)); - } - - 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(conf.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)) - .setOperationTimeout(conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_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(AsyncProcessTask task) - throws InterruptedIOException { - previousTimeout = task.getRpcTimeout(); - // We use results in tests to check things, so override to always save them. - AsyncProcessTask wrap = new AsyncProcessTask(task) { - @Override - public boolean getNeedResults() { - return true; - } - }; - return super.submit(wrap); - } - - @Override - protected RpcRetryingCaller createCaller( - CancellableRegionServerCallable callable, int rpcTimeout) { - callsCt.incrementAndGet(); - MultiServerCallable callable1 = (MultiServerCallable) callable; - final MultiResponse mr = createMultiResponse( - callable1.getMulti(), nbMultiResponse, nbActions, - new ResponseGenerator() { - @Override - public void addResponse(MultiResponse mr, byte[] regionName, Action a) { - if (Arrays.equals(FAILS, a.getAction().getRow())) { - mr.add(regionName, a.getOriginalIndex(), failure); - } else { - mr.add(regionName, a.getOriginalIndex(), success); - } - } - }); - - return new RpcRetryingCallerImpl(100, 500, 10, 9) { - @Override - public AbstractResponse callWithoutRetries(RetryingCallable callable, - int callTimeout) - throws IOException, RuntimeException { - try { - // sleep one second in order for threadpool to start another thread instead of reusing - // existing one. - Thread.sleep(1000); - } catch (InterruptedException e) { - // ignore error - } - return mr; - } - }; - } - - - } - - static class MyAsyncRequestFutureImpl extends AsyncRequestFutureImpl { - private final Map> heapSizesByServer = new HashMap<>(); - public MyAsyncRequestFutureImpl(AsyncProcessTask task, List actions, - long nonceGroup, AsyncProcess asyncProcess) { - super(task, actions, nonceGroup, asyncProcess); - } - - @Override - protected void updateStats(ServerName server, MultiResponse resp) { - // Do nothing for avoiding the NPE if we test the ClientBackofPolicy. - } - - Map> getRequestHeapSize() { - return heapSizesByServer; - } - - @Override - SingleServerRequestRunnable createSingleServerRequest( - MultiAction multiAction, int numAttempt, ServerName server, - Set callsInProgress) { - SingleServerRequestRunnable rq = new SingleServerRequestRunnable( - multiAction, numAttempt, server, callsInProgress); - List heapCount = heapSizesByServer.get(server); - if (heapCount == null) { - heapCount = new ArrayList<>(); - heapSizesByServer.put(server, heapCount); - } - heapCount.add(heapSizeOf(multiAction)); - return rq; - } - - private long heapSizeOf(MultiAction multiAction) { - return multiAction.actions.values().stream() - .flatMap(v -> v.stream()) - .map(action -> action.getAction()) - .filter(row -> row instanceof Mutation) - .mapToLong(row -> ((Mutation) row).heapSize()) - .sum(); - } - } - - static class CallerWithFailure extends RpcRetryingCallerImpl{ - - private final IOException e; - - public CallerWithFailure(IOException e) { - super(100, 500, 100, 9); - this.e = e; - } - - @Override - public AbstractResponse callWithoutRetries(RetryingCallable callable, - int callTimeout) - throws IOException, RuntimeException { - throw e; - } - } - - - static class AsyncProcessWithFailure extends MyAsyncProcess { - - private final IOException ioe; - - public AsyncProcessWithFailure(ConnectionImplementation hc, Configuration conf, - IOException ioe) { - super(hc, conf); - this.ioe = ioe; - serverTrackerTimeout = 1L; - } - - @Override - protected RpcRetryingCaller createCaller( - CancellableRegionServerCallable callable, int rpcTimeout) { - callsCt.incrementAndGet(); - return new CallerWithFailure(ioe); - } - } - - /** - * Make the backoff time always different on each call. - */ - static class MyClientBackoffPolicy implements ClientBackoffPolicy { - private final Map count = new HashMap<>(); - @Override - public long getBackoffTime(ServerName serverName, byte[] region, ServerStatistics stats) { - AtomicInteger inc = count.get(serverName); - if (inc == null) { - inc = new AtomicInteger(0); - count.put(serverName, inc); - } - return inc.getAndIncrement(); - } - } - - static class MyAsyncProcessWithReplicas extends MyAsyncProcess { - private Set failures = new TreeSet<>(new Bytes.ByteArrayComparator()); - private long primarySleepMs = 0, replicaSleepMs = 0; - private Map customPrimarySleepMs = new HashMap<>(); - private final AtomicLong replicaCalls = new AtomicLong(0); - - public void addFailures(RegionInfo... hris) { - for (RegionInfo hri : hris) { - failures.add(hri.getRegionName()); - } - } - - public long getReplicaCallCount() { - return replicaCalls.get(); - } - - public void setPrimaryCallDelay(ServerName server, long primaryMs) { - customPrimarySleepMs.put(server, primaryMs); - } - - public MyAsyncProcessWithReplicas(ConnectionImplementation hc, Configuration conf) { - super(hc, conf); - } - - public void setCallDelays(long primaryMs, long replicaMs) { - this.primarySleepMs = primaryMs; - this.replicaSleepMs = replicaMs; - } - - @Override - protected RpcRetryingCaller createCaller( - CancellableRegionServerCallable payloadCallable, int rpcTimeout) { - MultiServerCallable callable = (MultiServerCallable) payloadCallable; - final MultiResponse mr = createMultiResponse( - callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() { - @Override - public void addResponse(MultiResponse mr, byte[] regionName, Action a) { - if (failures.contains(regionName)) { - mr.add(regionName, a.getOriginalIndex(), failure); - } else { - boolean isStale = !RegionReplicaUtil.isDefaultReplica(a.getReplicaId()); - mr.add(regionName, a.getOriginalIndex(), - Result.create(new Cell[0], null, isStale)); - } - } - }); - // Currently AsyncProcess either sends all-replica, or all-primary request. - final boolean isDefault = RegionReplicaUtil.isDefaultReplica( - callable.getMulti().actions.values().iterator().next().iterator().next().getReplicaId()); - final ServerName server = ((MultiServerCallable)callable).getServerName(); - String debugMsg = "Call to " + server + ", primary=" + isDefault + " with " - + callable.getMulti().actions.size() + " entries: "; - for (byte[] region : callable.getMulti().actions.keySet()) { - debugMsg += "[" + Bytes.toStringBinary(region) + "], "; - } - LOG.debug(debugMsg); - if (!isDefault) { - replicaCalls.incrementAndGet(); - } - - return new RpcRetryingCallerImpl(100, 500, 10, 9) { - @Override - public MultiResponse callWithoutRetries(RetryingCallable callable, - int callTimeout) - throws IOException, RuntimeException { - long sleep = -1; - if (isDefault) { - Long customSleep = customPrimarySleepMs.get(server); - sleep = (customSleep == null ? primarySleepMs : customSleep.longValue()); - } else { - sleep = replicaSleepMs; - } - if (sleep != 0) { - try { - Thread.sleep(sleep); - } catch (InterruptedException e) { - } - } - return mr; - } - }; - } - } - - static MultiResponse createMultiResponse(final MultiAction multi, - AtomicInteger nbMultiResponse, AtomicInteger nbActions, ResponseGenerator gen) { - final MultiResponse mr = new MultiResponse(); - nbMultiResponse.incrementAndGet(); - for (Map.Entry> entry : multi.actions.entrySet()) { - byte[] regionName = entry.getKey(); - for (Action a : entry.getValue()) { - nbActions.incrementAndGet(); - gen.addResponse(mr, regionName, a); - } - } - return mr; - } - - private static interface ResponseGenerator { - void addResponse(final MultiResponse mr, byte[] regionName, Action a); - } - - /** - * Returns our async process. - */ - static class MyConnectionImpl extends ConnectionImplementation { - public static class TestRegistry extends DoNothingAsyncRegistry { - - public TestRegistry(Configuration conf) { - super(conf); - } - - @Override - public CompletableFuture getClusterId() { - return CompletableFuture.completedFuture("testClusterId"); - } - - @Override - public CompletableFuture getCurrentNrHRS() { - return CompletableFuture.completedFuture(1); - } - } - - final AtomicInteger nbThreads = new AtomicInteger(0); - - protected MyConnectionImpl(Configuration conf) throws IOException { - super(setupConf(conf), null, null); - } - - private static Configuration setupConf(Configuration conf) { - conf.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, TestRegistry.class, - AsyncRegistry.class); - return conf; - } - - @Override - public RegionLocations locateRegion(TableName tableName, - byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException { - return new RegionLocations(loc1); - } - - @Override - public boolean hasCellBlockSupport() { - return false; - } - } - - /** - * Returns our async process. - */ - static class MyConnectionImpl2 extends MyConnectionImpl { - List hrl; - final boolean usedRegions[]; - - protected MyConnectionImpl2(List hrl, Configuration conf) throws IOException { - super(conf); - this.hrl = hrl; - this.usedRegions = new boolean[hrl.size()]; - } - - @Override - public RegionLocations locateRegion(TableName tableName, - byte[] row, boolean useCache, boolean retry, int replicaId) throws IOException { - int i = 0; - for (HRegionLocation hr : hrl){ - if (Arrays.equals(row, hr.getRegionInfo().getStartKey())) { - usedRegions[i] = true; - return new RegionLocations(hr); - } - i++; - } - return null; - } - } - @Test - public void testListRowAccess() { - int count = 10; - List values = new LinkedList<>(); - for (int i = 0; i != count; ++i) { - values.add(String.valueOf(i)); - } - - ListRowAccess taker = new ListRowAccess(values); - assertEquals(count, taker.size()); - - int restoreCount = 0; - int takeCount = 0; - Iterator it = taker.iterator(); - while (it.hasNext()) { - String v = it.next(); - assertEquals(String.valueOf(takeCount), v); - ++takeCount; - it.remove(); - if (Math.random() >= 0.5) { - break; - } - } - assertEquals(count, taker.size() + takeCount); - - it = taker.iterator(); - while (it.hasNext()) { - String v = it.next(); - assertEquals(String.valueOf(takeCount), v); - ++takeCount; - it.remove(); - } - assertEquals(0, taker.size()); - assertEquals(count, takeCount); - } - private static long calculateRequestCount(long putSizePerServer, long maxHeapSizePerRequest) { - if (putSizePerServer <= maxHeapSizePerRequest) { - return 1; - } else if (putSizePerServer % maxHeapSizePerRequest == 0) { - return putSizePerServer / maxHeapSizePerRequest; - } else { - return putSizePerServer / maxHeapSizePerRequest + 1; - } - } - - @Test - public void testSubmitSameSizeOfRequest() throws Exception { - long writeBuffer = 2 * 1024 * 1024; - long putsHeapSize = writeBuffer; - doSubmitRequest(writeBuffer, putsHeapSize); - } - - @Test - public void testSubmitLargeRequestWithUnlimitedSize() throws Exception { - long maxHeapSizePerRequest = Long.MAX_VALUE; - long putsHeapSize = 2 * 1024 * 1024; - doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); - } - - @Test - public void testSubmitRandomSizeRequest() throws Exception { - Random rn = new Random(); - final long limit = 10 * 1024 * 1024; - final int requestCount = 1 + (int) (rn.nextDouble() * 3); - long n = rn.nextLong(); - if (n < 0) { - n = -n; - } else if (n == 0) { - n = 1; - } - long putsHeapSize = n % limit; - long maxHeapSizePerRequest = putsHeapSize / requestCount; - LOG.info("[testSubmitRandomSizeRequest] maxHeapSizePerRequest=" + maxHeapSizePerRequest + - ", putsHeapSize=" + putsHeapSize); - doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); - } - - @Test - public void testSubmitSmallRequest() throws Exception { - long maxHeapSizePerRequest = 2 * 1024 * 1024; - long putsHeapSize = 100; - doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); - } - - @Test - public void testSubmitLargeRequest() throws Exception { - long maxHeapSizePerRequest = 2 * 1024 * 1024; - long putsHeapSize = maxHeapSizePerRequest * 2; - doSubmitRequest(maxHeapSizePerRequest, putsHeapSize); - } - - private void doSubmitRequest(long maxHeapSizePerRequest, long putsHeapSize) throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - 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; - long putSizeSN2 = 0; - List puts = new ArrayList<>(); - while ((putSizeSN + putSizeSN2) <= putsHeapSize) { - Put put1 = new Put(DUMMY_BYTES_1); - put1.addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1); - Put put2 = new Put(DUMMY_BYTES_2); - put2.addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2); - Put put3 = new Put(DUMMY_BYTES_3); - put3.addColumn(DUMMY_BYTES_3, DUMMY_BYTES_3, DUMMY_BYTES_3); - putSizeSN += (put1.heapSize() + put2.heapSize()); - putSizeSN2 += put3.heapSize(); - puts.add(put1); - puts.add(put2); - puts.add(put3); - } - - int minCountSnRequest = (int) calculateRequestCount(putSizeSN, maxHeapSizePerRequest); - int minCountSn2Request = (int) calculateRequestCount(putSizeSN2, maxHeapSizePerRequest); - LOG.info("Total put count:" + puts.size() + ", putSizeSN:"+ putSizeSN - + ", putSizeSN2:" + putSizeSN2 - + ", maxHeapSizePerRequest:" + maxHeapSizePerRequest - + ", minCountSnRequest:" + minCountSnRequest - + ", minCountSn2Request:" + minCountSn2Request); - - MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - try (BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap)) { - mutator.mutate(puts); - mutator.flush(); - List reqs = ap.allReqs; - - int actualSnReqCount = 0; - int actualSn2ReqCount = 0; - for (AsyncRequestFuture req : reqs) { - if (!(req instanceof AsyncRequestFutureImpl)) { - continue; - } - MyAsyncRequestFutureImpl ars = (MyAsyncRequestFutureImpl) req; - if (ars.getRequestHeapSize().containsKey(sn)) { - ++actualSnReqCount; - } - if (ars.getRequestHeapSize().containsKey(sn2)) { - ++actualSn2ReqCount; - } - } - // If the server is busy, the actual count may be incremented. - assertEquals(true, minCountSnRequest <= actualSnReqCount); - assertEquals(true, minCountSn2Request <= actualSn2ReqCount); - Map sizePerServers = new HashMap<>(); - for (AsyncRequestFuture req : reqs) { - if (!(req instanceof AsyncRequestFutureImpl)) { - continue; - } - MyAsyncRequestFutureImpl ars = (MyAsyncRequestFutureImpl) req; - Map> requestHeapSize = ars.getRequestHeapSize(); - for (Map.Entry> entry : requestHeapSize.entrySet()) { - long sum = 0; - for (long size : entry.getValue()) { - assertEquals(true, size <= maxHeapSizePerRequest); - sum += size; - } - assertEquals(true, sum <= maxHeapSizePerRequest); - long value = sizePerServers.getOrDefault(entry.getKey(), 0L); - sizePerServers.put(entry.getKey(), value + sum); - } - } - assertEquals(true, sizePerServers.containsKey(sn)); - assertEquals(true, sizePerServers.containsKey(sn2)); - assertEquals(false, sizePerServers.containsKey(sn3)); - assertEquals(putSizeSN, (long) sizePerServers.get(sn)); - assertEquals(putSizeSN2, (long) sizePerServers.get(sn2)); - } - // restore config. - 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 { - ConnectionImplementation hc = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); - - List puts = new ArrayList<>(1); - puts.add(createPut(1, true)); - - ap.submit(null, DUMMY_TABLE, puts, false, null, false); - Assert.assertTrue(puts.isEmpty()); - } - - @Test - public void testSubmitWithCB() throws Exception { - ConnectionImplementation hc = createConnectionImpl(); - final AtomicInteger updateCalled = new AtomicInteger(0); - Batch.Callback cb = new Batch.Callback() { - @Override - public void update(byte[] region, byte[] row, Object result) { - updateCalled.incrementAndGet(); - } - }; - MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); - - List puts = new ArrayList<>(1); - puts.add(createPut(1, true)); - - final AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, cb, false); - Assert.assertTrue(puts.isEmpty()); - ars.waitUntilDone(); - Assert.assertEquals(1, updateCalled.get()); - } - - @Test - public void testSubmitBusyRegion() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - 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<>(1); - puts.add(createPut(1, true)); - - for (int i = 0; i != controller.maxConcurrentTasksPerRegion; ++i) { - ap.incTaskCounters(Collections.singleton(hri1.getRegionName()), sn); - } - ap.submit(null, DUMMY_TABLE, puts, false, null, false); - Assert.assertEquals(puts.size(), 1); - - ap.decTaskCounters(Collections.singleton(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 { - ConnectionImplementation conn = createConnectionImpl(); - 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<>(4); - puts.add(createPut(1, true)); - puts.add(createPut(3, true)); // <== this one won't be taken, the rs is busy - puts.add(createPut(1, true)); // <== this one will make it, the region is already in - puts.add(createPut(2, true)); // <== new region, but the rs is ok - - ap.submit(null, DUMMY_TABLE, puts, false, null, false); - Assert.assertEquals(" puts=" + puts, 1, puts.size()); - - 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(createConnectionImpl(), CONF); - - List puts = new ArrayList<>(1); - Put p = createPut(1, false); - puts.add(p); - - AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); - Assert.assertEquals(0, puts.size()); - ars.waitUntilDone(); - verifyResult(ars, false); - Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); - - Assert.assertEquals(1, ars.getErrors().exceptions.size()); - Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0), - failure.equals(ars.getErrors().exceptions.get(0))); - Assert.assertTrue("was: " + ars.getErrors().exceptions.get(0), - failure.equals(ars.getErrors().exceptions.get(0))); - - Assert.assertEquals(1, ars.getFailedOperations().size()); - Assert.assertTrue("was: " + ars.getFailedOperations().get(0), - p.equals(ars.getFailedOperations().get(0))); - } - - - @Test - public void testSubmitTrue() throws IOException { - ConnectionImplementation conn = createConnectionImpl(); - final 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.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); - - Thread t = new Thread(){ - @Override - public void run(){ - Threads.sleep(1000); - Assert.assertFalse(checkPoint.get()); // TODO: this is timing-dependent - ai.decrementAndGet(); - controller.tasksInProgress.decrementAndGet(); - checkPoint2.set(true); - } - }; - - List puts = new ArrayList<>(1); - Put p = createPut(1, true); - puts.add(p); - - ap.submit(null, DUMMY_TABLE, puts, false, null, false); - Assert.assertFalse(puts.isEmpty()); - - t.start(); - - ap.submit(null, DUMMY_TABLE, puts, true, null, false); - Assert.assertTrue(puts.isEmpty()); - - checkPoint.set(true); - 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(createConnectionImpl(), CONF); - - List puts = new ArrayList<>(3); - puts.add(createPut(1, false)); - puts.add(createPut(1, true)); - puts.add(createPut(1, true)); - - AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); - Assert.assertTrue(puts.isEmpty()); - ars.waitUntilDone(); - verifyResult(ars, false, true, true); - Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); - ap.callsCt.set(0); - Assert.assertEquals(1, ars.getErrors().actions.size()); - - puts.add(createPut(1, true)); - // Wait for AP to be free. While ars might have the result, ap counters are decreased later. - ap.waitForMaximumCurrentTasks(0, null); - ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); - Assert.assertEquals(0, puts.size()); - ars.waitUntilDone(); - Assert.assertEquals(1, ap.callsCt.get()); - verifyResult(ars, true); - } - - @Test - public void testFlush() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createConnectionImpl(), CONF); - - List puts = new ArrayList<>(3); - puts.add(createPut(1, false)); - puts.add(createPut(1, true)); - puts.add(createPut(1, true)); - - AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, null, true); - ars.waitUntilDone(); - verifyResult(ars, false, true, true); - Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); - - Assert.assertEquals(1, ars.getFailedOperations().size()); - } - - @Test - public void testTaskCountWithoutClientBackoffPolicy() throws IOException, InterruptedException { - ConnectionImplementation hc = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); - testTaskCount(ap); - } - - @Test - public void testTaskCountWithClientBackoffPolicy() throws IOException, InterruptedException { - Configuration copyConf = new Configuration(CONF); - copyConf.setBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, true); - MyClientBackoffPolicy bp = new MyClientBackoffPolicy(); - ConnectionImplementation conn = createConnectionImpl(); - 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); - testTaskCount(ap); - if (defaultClazz != null) { - conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY, - defaultClazz); - } - } - - 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)); - puts.add(createPut(2, true)); - puts.add(createPut(3, true)); - } - ap.submit(null, DUMMY_TABLE, puts, true, null, false); - ap.waitForMaximumCurrentTasks(0, null); - // More time to wait if there are incorrect task count. - TimeUnit.SECONDS.sleep(1); - assertEquals(0, controller.tasksInProgress.get()); - for (AtomicInteger count : controller.taskCounterPerRegion.values()) { - assertEquals(0, count.get()); - } - for (AtomicInteger count : controller.taskCounterPerServer.values()) { - assertEquals(0, count.get()); - } - } - - @Test - public void testMaxTask() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - 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); - SimpleRequestController controller = (SimpleRequestController) ap.requestController; - - - for (int i = 0; i < 1000; i++) { - ap.incTaskCounters(Collections.singleton(Bytes.toBytes("dummy")), sn); - } - - final Thread myThread = Thread.currentThread(); - - Thread t = new Thread() { - @Override - public void run() { - Threads.sleep(2000); - myThread.interrupt(); - } - }; - - List puts = new ArrayList<>(1); - puts.add(createPut(1, true)); - - t.start(); - - try { - ap.submit(null, DUMMY_TABLE, puts, false, null, false); - Assert.fail("We should have been interrupted."); - } catch (InterruptedIOException expected) { - } - - final long sleepTime = 2000; - - Thread t2 = new Thread() { - @Override - public void run() { - Threads.sleep(sleepTime); - while (controller.tasksInProgress.get() > 0) { - ap.decTaskCounters(Collections.singleton(Bytes.toBytes("dummy")), sn); - } - } - }; - t2.start(); - - long start = System.currentTimeMillis(); - ap.submit(null, DUMMY_TABLE, new ArrayList<>(), false, null, false); - long end = System.currentTimeMillis(); - - //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 ConnectionImplementation createConnectionImpl() throws IOException { - ConnectionImplementation hc = createConnectionImplCommon(); - setMockLocation(hc, DUMMY_BYTES_1, new RegionLocations(loc1)); - setMockLocation(hc, DUMMY_BYTES_2, new RegionLocations(loc2)); - setMockLocation(hc, DUMMY_BYTES_3, new RegionLocations(loc3)); - Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), - Mockito.anyBoolean())).thenReturn(Arrays.asList(loc1, loc2, loc3)); - setMockLocation(hc, FAILS, new RegionLocations(loc2)); - return hc; - } - - private ConnectionImplementation createConnectionImplWithReplicas() throws IOException { - ConnectionImplementation hc = createConnectionImplCommon(); - setMockLocation(hc, DUMMY_BYTES_1, hrls1); - setMockLocation(hc, DUMMY_BYTES_2, hrls2); - setMockLocation(hc, DUMMY_BYTES_3, hrls3); - List locations = new ArrayList<>(); - for (HRegionLocation loc : hrls1.getRegionLocations()) { - locations.add(loc); - } - for (HRegionLocation loc : hrls2.getRegionLocations()) { - locations.add(loc); - } - for (HRegionLocation loc : hrls3.getRegionLocations()) { - locations.add(loc); - } - Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), - Mockito.anyBoolean())).thenReturn(locations); - return hc; - } - - private static void setMockLocation(ConnectionImplementation hc, byte[] row, - RegionLocations result) throws IOException { - Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(), - Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result); - Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(), - Mockito.anyBoolean())).thenReturn(result); - } - - private ConnectionImplementation createConnectionImplCommon() { - ConnectionImplementation hc = Mockito.mock(ConnectionImplementation.class); - 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.getConnectionConfiguration()).thenReturn(CONNECTION_CONFIG); - return hc; - } - - @Test - public void testHTablePutSuccess() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap); - - Put put = createPut(1, true); - - Assert.assertEquals(conn.getConnectionConfiguration().getWriteBufferSize(), - ht.getWriteBufferSize()); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - ht.mutate(put); - ht.flush(); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - } - - @Test - public void testSettingWriteBufferPeriodicFlushParameters() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); - - checkPeriodicFlushParameters(conn, ap, - 1234, 1234, - 1234, 1234); - checkPeriodicFlushParameters(conn, ap, - 0, 0, - 0, BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); - checkPeriodicFlushParameters(conn, ap, - -1234, 0, - -1234, BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); - checkPeriodicFlushParameters(conn, ap, - 1, 1, - 1, BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS); - } - - private void checkPeriodicFlushParameters(ConnectionImplementation conn, MyAsyncProcess ap, - long setTO, long expectTO, long setTT, long expectTT) { - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - - // The BufferedMutatorParams does nothing with the value - bufferParam.setWriteBufferPeriodicFlushTimeoutMs(setTO); - bufferParam.setWriteBufferPeriodicFlushTimerTickMs(setTT); - Assert.assertEquals(setTO, bufferParam.getWriteBufferPeriodicFlushTimeoutMs()); - Assert.assertEquals(setTT, bufferParam.getWriteBufferPeriodicFlushTimerTickMs()); - - // The BufferedMutatorImpl corrects illegal values (indirect via BufferedMutatorParams) - BufferedMutatorImpl ht1 = new BufferedMutatorImpl(conn, bufferParam, ap); - Assert.assertEquals(expectTO, ht1.getWriteBufferPeriodicFlushTimeoutMs()); - Assert.assertEquals(expectTT, ht1.getWriteBufferPeriodicFlushTimerTickMs()); - - // The BufferedMutatorImpl corrects illegal values (direct via setter) - BufferedMutatorImpl ht2 = - new BufferedMutatorImpl(conn, createBufferedMutatorParams(ap, DUMMY_TABLE), ap); - ht2.setWriteBufferPeriodicFlush(setTO, setTT); - Assert.assertEquals(expectTO, ht2.getWriteBufferPeriodicFlushTimeoutMs()); - Assert.assertEquals(expectTT, ht2.getWriteBufferPeriodicFlushTimerTickMs()); - - } - - @Test - public void testWriteBufferPeriodicFlushTimeoutMs() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - - bufferParam.setWriteBufferPeriodicFlushTimeoutMs(1); // Flush ASAP - bufferParam.setWriteBufferPeriodicFlushTimerTickMs(1); // Check every 100ms - bufferParam.writeBufferSize(10000); // Write buffer set to much larger than the single record - - BufferedMutatorImpl ht = new BufferedMutatorImpl(conn, bufferParam, ap); - - // Verify if BufferedMutator has the right settings. - Assert.assertEquals(10000, ht.getWriteBufferSize()); - Assert.assertEquals(1, ht.getWriteBufferPeriodicFlushTimeoutMs()); - Assert.assertEquals(BufferedMutator.MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS, - ht.getWriteBufferPeriodicFlushTimerTickMs()); - - Put put = createPut(1, true); - - Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - - // ----- Insert, flush immediately, MUST NOT flush automatically - ht.mutate(put); - ht.flush(); - - Thread.sleep(1000); - Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - - // ----- Insert, NO flush, MUST flush automatically - ht.mutate(put); - Assert.assertEquals(0, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0); - - // The timerTick should fire every 100ms, so after twice that we must have - // seen at least 1 tick and we should see an automatic flush - Thread.sleep(200); - Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - - // Ensure it does not flush twice - Thread.sleep(200); - Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - - // ----- DISABLE AUTO FLUSH, Insert, NO flush, MUST NOT flush automatically - ht.disableWriteBufferPeriodicFlush(); - ht.mutate(put); - Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0); - - // Wait for at least 1 timerTick, we should see NO flushes. - Thread.sleep(200); - Assert.assertEquals(1, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertTrue(ht.getCurrentWriteBufferSize() > 0); - - // Reenable periodic flushing, a flush seems to take about 1 second - // so we wait for 2 seconds and it should have finished the flush. - ht.setWriteBufferPeriodicFlush(1, 100); - Thread.sleep(2000); - Assert.assertEquals(2, ht.getExecutedWriteBufferPeriodicFlushes()); - Assert.assertEquals(0, ht.getCurrentWriteBufferSize()); - } - - - @Test - public void testBufferedMutatorImplWithSharedPool() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - BufferedMutator ht = new BufferedMutatorImpl(conn, bufferParam, ap); - - ht.close(); - assertFalse(ap.service.isShutdown()); - } - - @Test - public void testFailedPutAndNewPut() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE) - .writeBufferSize(0); - BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - - Put p = createPut(1, false); - try { - mutator.mutate(p); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - // Let's do all the retries. - ap.waitForMaximumCurrentTasks(0, null); - Assert.assertEquals(0, mutator.size()); - - // There is no global error so the new put should not fail - mutator.mutate(createPut(1, true)); - Assert.assertEquals("the put should not been inserted.", 0, mutator.size()); - } - - @SuppressWarnings("SelfComparison") - @Test - public void testAction() { - Action action_0 = new Action(new Put(Bytes.toBytes("abc")), 10); - Action action_1 = new Action(new Put(Bytes.toBytes("ccc")), 10); - Action action_2 = new Action(new Put(Bytes.toBytes("ccc")), 10); - Action action_3 = new Action(new Delete(Bytes.toBytes("ccc")), 10); - assertFalse(action_0.equals(action_1)); - assertTrue(action_0.equals(action_0)); - assertTrue(action_1.equals(action_2)); - assertTrue(action_2.equals(action_1)); - assertFalse(action_0.equals(new Put(Bytes.toBytes("abc")))); - assertTrue(action_2.equals(action_3)); - assertFalse(action_0.equals(action_3)); - assertEquals(0, action_0.compareTo(action_0)); - assertTrue(action_0.compareTo(action_1) < 0); - assertTrue(action_1.compareTo(action_0) > 0); - assertEquals(0, action_1.compareTo(action_2)); - } - - @Test - public void testBatch() throws IOException, InterruptedException { - ConnectionImplementation conn = new MyConnectionImpl(CONF); - HTable ht = (HTable) conn.getTable(DUMMY_TABLE); - ht.multiAp = new MyAsyncProcess(conn, CONF); - - List puts = new ArrayList<>(7); - puts.add(createPut(1, true)); - puts.add(createPut(1, true)); - puts.add(createPut(1, true)); - puts.add(createPut(1, true)); - puts.add(createPut(1, false)); // <=== the bad apple, position 4 - puts.add(createPut(1, true)); - puts.add(createPut(1, false)); // <=== another bad apple, position 6 - - Object[] res = new Object[puts.size()]; - try { - ht.batch(puts, res); - Assert.fail(); - } catch (RetriesExhaustedException expected) { - } - - Assert.assertEquals(success, res[0]); - Assert.assertEquals(success, res[1]); - Assert.assertEquals(success, res[2]); - Assert.assertEquals(success, res[3]); - Assert.assertEquals(failure, res[4]); - Assert.assertEquals(success, res[5]); - Assert.assertEquals(failure, res[6]); - } - @Test - public void testErrorsServers() throws IOException { - Configuration configuration = new Configuration(CONF); - ConnectionImplementation conn = new MyConnectionImpl(configuration); - MyAsyncProcess ap = new MyAsyncProcess(conn, configuration); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - configuration.setBoolean(ConnectionImplementation.RETRIES_BY_SERVER_KEY, true); - - Assert.assertNotNull(ap.createServerErrorTracker()); - Assert.assertTrue(ap.serverTrackerTimeout > 200L); - ap.serverTrackerTimeout = 1L; - - Put p = createPut(1, false); - mutator.mutate(p); - - try { - mutator.flush(); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - // Checking that the ErrorsServers came into play and didn't make us stop immediately - Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); - } - - @Test - public void testReadAndWriteTimeout() throws IOException { - final long readTimeout = 10 * 1000; - final long writeTimeout = 20 * 1000; - Configuration copyConf = new Configuration(CONF); - copyConf.setLong(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, readTimeout); - copyConf.setLong(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, writeTimeout); - ConnectionImplementation conn = new MyConnectionImpl(copyConf); - MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf); - try (HTable ht = (HTable) conn.getTable(DUMMY_TABLE)) { - ht.multiAp = ap; - List gets = new LinkedList<>(); - gets.add(new Get(DUMMY_BYTES_1)); - gets.add(new Get(DUMMY_BYTES_2)); - try { - ht.get(gets); - } catch (ClassCastException e) { - // No result response on this test. - } - assertEquals(readTimeout, ap.previousTimeout); - ap.previousTimeout = -1; - - try { - ht.exists(gets); - } catch (ClassCastException e) { - // No result response on this test. - } - assertEquals(readTimeout, ap.previousTimeout); - ap.previousTimeout = -1; - - List deletes = new LinkedList<>(); - deletes.add(new Delete(DUMMY_BYTES_1)); - deletes.add(new Delete(DUMMY_BYTES_2)); - ht.delete(deletes); - assertEquals(writeTimeout, ap.previousTimeout); - } - } - - @Test - public void testErrors() throws IOException { - ConnectionImplementation 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(ap.createServerErrorTracker()); - - Put p = createPut(1, true); - mutator.mutate(p); - - try { - mutator.flush(); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - // Checking that the ErrorsServers came into play and didn't make us stop immediately - Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); - } - - - @Test - public void testCallQueueTooLarge() throws IOException { - ConnectionImplementation 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); - - try { - mutator.flush(); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - // Checking that the ErrorsServers came into play and didn't make us stop immediately - Assert.assertEquals(NB_RETRIES + 1, ap.callsCt.get()); - } - /** - * This test simulates multiple regions on 2 servers. We should have 2 multi requests and - * 2 threads: 1 per server, this whatever the number of regions. - */ - @Test - public void testThreadCreation() throws Exception { - final int NB_REGS = 100; - List hrls = new ArrayList<>(NB_REGS); - List gets = new ArrayList<>(NB_REGS); - for (int i = 0; i < NB_REGS; i++) { - HRegionInfo hri = new HRegionInfo( - DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i); - HRegionLocation hrl = new HRegionLocation(hri, i % 2 == 0 ? sn : sn2); - hrls.add(hrl); - - Get get = new Get(Bytes.toBytes(i * 10L)); - gets.add(get); - } - - MyConnectionImpl2 con = new MyConnectionImpl2(hrls, CONF); - MyAsyncProcess ap = new MyAsyncProcess(con, CONF, con.nbThreads); - HTable ht = (HTable) con.getTable(DUMMY_TABLE, ap.service); - ht.multiAp = ap; - ht.batch(gets, null); - - Assert.assertEquals(NB_REGS, ap.nbActions.get()); - Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get()); - Assert.assertEquals("1 thread per server", 2, con.nbThreads.get()); - - int nbReg = 0; - for (int i =0; i blockingqueue) { - super(coreThreads, maxThreads, keepAliveTime, timeunit, blockingqueue); - } - - @Override - public Future submit(Runnable runnable) { - throw new OutOfMemoryError("OutOfMemory error thrown by means"); - } - } - - static class AsyncProcessForThrowableCheck extends AsyncProcess { - public AsyncProcessForThrowableCheck(ConnectionImplementation hc, Configuration conf) { - super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf)); - } - } - - @Test - public void testUncheckedException() throws Exception { - // Test the case pool.submit throws unchecked exception - ConnectionImplementation hc = createConnectionImpl(); - MyThreadPoolExecutor myPool = - new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, - new LinkedBlockingQueue<>(200)); - AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, CONF); - - List puts = new ArrayList<>(1); - puts.add(createPut(1, true)); - 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 and make sure we could use a special pause setting when retry with - * CallQueueTooBigException, see HBASE-17114 - * @throws Exception if unexpected error happened during test - */ - @Test - public void testRetryPauseWithCallQueueTooBigException() throws Exception { - 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); - ConnectionImplementation conn = new MyConnectionImpl(myConf); - AsyncProcessWithFailure ap = - new AsyncProcessWithFailure(conn, myConf, new CallQueueTooBigException()); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - - Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); - - Put p = createPut(1, true); - mutator.mutate(p); - - long startTime = System.currentTimeMillis(); - try { - mutator.flush(); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - long actualSleep = System.currentTimeMillis() - startTime; - long expectedSleep = 0L; - for (int i = 0; i < retries; i++) { - expectedSleep += ConnectionUtils.getPauseTime(specialPause, i); - // Prevent jitter in ConcurrentMapUtils#getPauseTime to affect result - actualSleep += (long) (specialPause * 0.01f); - } - LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms"); - Assert.assertTrue("Expected to sleep " + expectedSleep + " but actually " + actualSleep + "ms", - actualSleep >= expectedSleep); - - // check and confirm normal IOE will use the normal pause - final long normalPause = - myConf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE); - ap = new AsyncProcessWithFailure(conn, myConf, new IOException()); - bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); - mutator.mutate(p); - startTime = System.currentTimeMillis(); - try { - mutator.flush(); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - actualSleep = System.currentTimeMillis() - startTime; - expectedSleep = 0L; - for (int i = 0; i < retries; i++) { - expectedSleep += ConnectionUtils.getPauseTime(normalPause, i); - } - // plus an additional pause to balance the program execution time - expectedSleep += normalPause; - LOG.debug("Expected to sleep " + expectedSleep + "ms, actually slept " + actualSleep + "ms"); - Assert.assertTrue("Slept for too long: " + actualSleep + "ms", actualSleep <= expectedSleep); - } - - @Test - public void testRetryWithExceptionClearsMetaCache() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - Configuration myConf = conn.getConfiguration(); - myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0); - - AsyncProcessWithFailure ap = - new AsyncProcessWithFailure(conn, myConf, new RegionOpeningException("test")); - BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE); - BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap); - - Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker()); - - Assert.assertEquals( - conn.locateRegion(DUMMY_TABLE, DUMMY_BYTES_1, true, true).toString(), - new RegionLocations(loc1).toString()); - - Mockito.verify(conn, Mockito.times(0)).clearCaches(Mockito.any()); - - Put p = createPut(1, true); - mutator.mutate(p); - - try { - mutator.flush(); - Assert.fail(); - } catch (RetriesExhaustedWithDetailsException expected) { - assertEquals(1, expected.getNumExceptions()); - assertTrue(expected.getRow(0) == p); - } - - Mockito.verify(conn, Mockito.times(1)).clearCaches(loc1.getServerName()); - } - - @Test - public void testQueueRowAccess() throws Exception { - ConnectionImplementation conn = createConnectionImpl(); - BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null, - new BufferedMutatorParams(DUMMY_TABLE).writeBufferSize(100000)); - Put p0 = new Put(DUMMY_BYTES_1).addColumn(DUMMY_BYTES_1, DUMMY_BYTES_1, DUMMY_BYTES_1); - Put p1 = new Put(DUMMY_BYTES_2).addColumn(DUMMY_BYTES_2, DUMMY_BYTES_2, DUMMY_BYTES_2); - mutator.mutate(p0); - BufferedMutatorImpl.QueueRowAccess ra0 = mutator.createQueueRowAccess(); - // QueueRowAccess should take all undealt mutations - assertEquals(0, mutator.size()); - mutator.mutate(p1); - assertEquals(1, mutator.size()); - BufferedMutatorImpl.QueueRowAccess ra1 = mutator.createQueueRowAccess(); - // QueueRowAccess should take all undealt mutations - assertEquals(0, mutator.size()); - assertEquals(1, ra0.size()); - assertEquals(1, ra1.size()); - Iterator iter0 = ra0.iterator(); - Iterator iter1 = ra1.iterator(); - assertTrue(iter0.hasNext()); - assertTrue(iter1.hasNext()); - // the next() will poll the mutation from inner buffer and update the buffer count - assertTrue(iter0.next() == p0); - assertEquals(1, mutator.getUnflushedSize()); - assertEquals(p1.heapSize(), mutator.getCurrentWriteBufferSize()); - assertTrue(iter1.next() == p1); - assertEquals(0, mutator.getUnflushedSize()); - assertEquals(0, mutator.getCurrentWriteBufferSize()); - assertFalse(iter0.hasNext()); - assertFalse(iter1.hasNext()); - // ra0 doest handle the mutation so the mutation won't be pushed back to buffer - iter0.remove(); - ra0.close(); - assertEquals(0, mutator.size()); - assertEquals(0, mutator.getUnflushedSize()); - assertEquals(0, mutator.getCurrentWriteBufferSize()); - // ra1 doesn't handle the mutation so the mutation will be pushed back to buffer - ra1.close(); - assertEquals(1, mutator.size()); - assertEquals(1, mutator.getUnflushedSize()); - assertEquals(p1.heapSize(), mutator.getCurrentWriteBufferSize()); - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java deleted file mode 100644 index 2c24aaa7be39..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java +++ /dev/null @@ -1,252 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HConstants; -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.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; - -/** - * The purpose of this test is to make sure the region exception won't corrupt the results - * of batch. The prescription is shown below. - * 1) honor the action result rather than region exception. If the action have both of true result - * and region exception, the action is fine as the exception is caused by other actions - * which are in the same region. - * 2) honor the action exception rather than region exception. If the action have both of action - * exception and region exception, we deal with the action exception only. If we also - * handle the region exception for the same action, it will introduce the negative count of - * actions in progress. The AsyncRequestFuture#waitUntilDone will block forever. - * - * This bug can be reproduced by real use case. see TestMalformedCellFromClient(in branch-1.4+). - * It uses the batch of RowMutations to present the bug. Given that the batch of RowMutations is - * only supported by branch-1.4+, perhaps the branch-1.3 and branch-1.2 won't encounter this issue. - * We still backport the fix to branch-1.3 and branch-1.2 in case we ignore some write paths. - */ -@Category({ ClientTests.class, SmallTests.class }) -public class TestAsyncProcessWithRegionException { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestAsyncProcessWithRegionException.class); - - private static final Result EMPTY_RESULT = Result.create(null, true); - private static final IOException IOE = new IOException("YOU CAN'T PASS"); - private static final Configuration CONF = new Configuration(); - private static final TableName DUMMY_TABLE = TableName.valueOf("DUMMY_TABLE"); - private static final byte[] GOOD_ROW = Bytes.toBytes("GOOD_ROW"); - private static final byte[] BAD_ROW = Bytes.toBytes("BAD_ROW"); - private static final byte[] BAD_ROW_WITHOUT_ACTION_EXCEPTION = - Bytes.toBytes("BAD_ROW_WITHOUT_ACTION_EXCEPTION"); - private static final byte[] FAMILY = Bytes.toBytes("FAMILY"); - private static final ServerName SERVER_NAME = ServerName.valueOf("s1,1,1"); - private static final RegionInfo REGION_INFO = - RegionInfoBuilder.newBuilder(DUMMY_TABLE) - .setStartKey(HConstants.EMPTY_START_ROW) - .setEndKey(HConstants.EMPTY_END_ROW) - .setSplit(false) - .setRegionId(1) - .build(); - - private static final HRegionLocation REGION_LOCATION = - new HRegionLocation(REGION_INFO, SERVER_NAME); - - @BeforeClass - public static void setUpBeforeClass() { - // disable the retry - CONF.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0); - } - - @Test - public void testSuccessivePut() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF); - - List puts = new ArrayList<>(1); - puts.add(new Put(GOOD_ROW).addColumn(FAMILY, FAMILY, FAMILY)); - final int expectedSize = puts.size(); - AsyncRequestFuture arf = ap.submit(DUMMY_TABLE, puts); - arf.waitUntilDone(); - Object[] result = arf.getResults(); - assertEquals(expectedSize, result.length); - for (Object r : result) { - assertEquals(Result.class, r.getClass()); - } - assertTrue(puts.isEmpty()); - assertActionsInProgress(arf); - } - - @Test - public void testFailedPut() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF); - - List puts = new ArrayList<>(2); - puts.add(new Put(GOOD_ROW).addColumn(FAMILY, FAMILY, FAMILY)); - // this put should fail - puts.add(new Put(BAD_ROW).addColumn(FAMILY, FAMILY, FAMILY)); - final int expectedSize = puts.size(); - - AsyncRequestFuture arf = ap.submit(DUMMY_TABLE, puts); - arf.waitUntilDone(); - // There is a failed puts - assertError(arf, 1); - Object[] result = arf.getResults(); - assertEquals(expectedSize, result.length); - assertEquals(Result.class, result[0].getClass()); - assertTrue(result[1] instanceof IOException); - assertTrue(puts.isEmpty()); - assertActionsInProgress(arf); - } - - @Test - public void testFailedPutWithoutActionException() throws Exception { - MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF); - - List puts = new ArrayList<>(3); - puts.add(new Put(GOOD_ROW).addColumn(FAMILY, FAMILY, FAMILY)); - // this put should fail - puts.add(new Put(BAD_ROW).addColumn(FAMILY, FAMILY, FAMILY)); - // this put should fail, and it won't have action exception - puts.add(new Put(BAD_ROW_WITHOUT_ACTION_EXCEPTION).addColumn(FAMILY, FAMILY, FAMILY)); - final int expectedSize = puts.size(); - - AsyncRequestFuture arf = ap.submit(DUMMY_TABLE, puts); - arf.waitUntilDone(); - // There are two failed puts - assertError(arf, 2); - Object[] result = arf.getResults(); - assertEquals(expectedSize, result.length); - assertEquals(Result.class, result[0].getClass()); - assertTrue(result[1] instanceof IOException); - assertTrue(result[2] instanceof IOException); - assertTrue(puts.isEmpty()); - assertActionsInProgress(arf); - } - - private static void assertError(AsyncRequestFuture arf, int expectedCountOfFailure) { - assertTrue(arf.hasError()); - RetriesExhaustedWithDetailsException e = arf.getErrors(); - List errors = e.getCauses(); - assertEquals(expectedCountOfFailure, errors.size()); - for (Throwable t : errors) { - assertTrue(t instanceof IOException); - } - } - - private static void assertActionsInProgress(AsyncRequestFuture arf) { - if (arf instanceof AsyncRequestFutureImpl) { - assertEquals(0, ((AsyncRequestFutureImpl) arf).getNumberOfActionsInProgress()); - } - } - - private static ConnectionImplementation createHConnection() throws IOException { - ConnectionImplementation hc = Mockito.mock(ConnectionImplementation.class); - 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.getConnectionConfiguration()).thenReturn(new ConnectionConfiguration(CONF)); - setMockLocation(hc, GOOD_ROW, new RegionLocations(REGION_LOCATION)); - setMockLocation(hc, BAD_ROW, new RegionLocations(REGION_LOCATION)); - Mockito - .when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), Mockito.anyBoolean())) - .thenReturn(Collections.singletonList(REGION_LOCATION)); - return hc; - } - - private static void setMockLocation(ConnectionImplementation hc, byte[] row, - RegionLocations result) throws IOException { - Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(), - Mockito.anyBoolean(), Mockito.anyInt())).thenReturn(result); - Mockito.when(hc.locateRegion(Mockito.eq(DUMMY_TABLE), Mockito.eq(row), Mockito.anyBoolean(), - Mockito.anyBoolean())).thenReturn(result); - } - - private static class MyAsyncProcess extends AsyncProcess { - private final ExecutorService service = Executors.newFixedThreadPool(5); - - MyAsyncProcess(ConnectionImplementation hc, Configuration conf) { - super(hc, conf, new RpcRetryingCallerFactory(conf), new RpcControllerFactory(conf)); - } - - public AsyncRequestFuture submit(TableName tableName, List rows) - throws InterruptedIOException { - return submit(AsyncProcessTask.newBuilder() - .setPool(service) - .setTableName(tableName) - .setRowAccess(rows) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.NORMAL) - .setNeedResults(true) - .setRpcTimeout(HConstants.DEFAULT_HBASE_RPC_TIMEOUT) - .setOperationTimeout(HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT) - .build()); - } - - @Override - protected RpcRetryingCaller createCaller( - CancellableRegionServerCallable callable, int rpcTimeout) { - MultiServerCallable callable1 = (MultiServerCallable) callable; - MultiResponse mr = new MultiResponse(); - callable1.getMulti().actions.forEach((regionName, actions) -> { - actions.forEach(action -> { - if (Bytes.equals(action.getAction().getRow(), GOOD_ROW)) { - mr.add(regionName, action.getOriginalIndex(), EMPTY_RESULT); - } else if (Bytes.equals(action.getAction().getRow(), BAD_ROW)) { - mr.add(regionName, action.getOriginalIndex(), IOE); - } - }); - }); - mr.addException(REGION_INFO.getRegionName(), IOE); - return new RpcRetryingCallerImpl(100, 500, 0, 9) { - @Override - public AbstractResponse callWithoutRetries(RetryingCallable callable, - int callTimeout) { - try { - // sleep one second in order for threadpool to start another thread instead of reusing - // existing one. - Thread.sleep(1000); - } catch (InterruptedException e) { - // pass - } - return mr; - } - }; - } - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java deleted file mode 100644 index fd3a4f8deb85..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ /dev/null @@ -1,820 +0,0 @@ -/** - * 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 static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import java.net.SocketTimeoutException; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.Random; -import java.util.SortedMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.NotImplementedException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.RegionTooBusyException; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch; -import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; - -/** - * Test client behavior w/o setting up a cluster. - * Mock up cluster emissions. - */ -@Category({ClientTests.class, SmallTests.class}) -public class TestClientNoCluster extends Configured implements Tool { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestClientNoCluster.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestClientNoCluster.class); - private Configuration conf; - public static final ServerName META_SERVERNAME = - ServerName.valueOf("meta.example.org", 16010, 12345); - - @Before - public void setUp() throws Exception { - this.conf = HBaseConfiguration.create(); - // Run my Connection overrides. Use my little ConnectionImplementation below which - // allows me insert mocks and also use my Registry below rather than the default zk based - // one so tests run faster and don't have zk dependency. - this.conf.set("hbase.client.registry.impl", SimpleRegistry.class.getName()); - SimpleRegistry.setMetaHost(conf, META_SERVERNAME); - } - - /** - * Remove the @Ignore to try out timeout and retry asettings - * @throws IOException - */ - @Ignore - @Test - public void testTimeoutAndRetries() throws IOException { - Configuration localConfig = HBaseConfiguration.create(this.conf); - // This override mocks up our exists/get call to throw a RegionServerStoppedException. - localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); - Connection connection = ConnectionFactory.createConnection(localConfig); - Table table = connection.getTable(TableName.META_TABLE_NAME); - Throwable t = null; - LOG.info("Start"); - try { - // An exists call turns into a get w/ a flag. - table.exists(new Get(Bytes.toBytes("abc"))); - } catch (SocketTimeoutException e) { - // I expect this exception. - LOG.info("Got expected exception", e); - t = e; - } catch (RetriesExhaustedException e) { - // This is the old, unwanted behavior. If we get here FAIL!!! - fail(); - } finally { - table.close(); - } - connection.close(); - LOG.info("Stop"); - assertTrue(t != null); - } - - /** - * Test that operation timeout prevails over rpc default timeout and retries, etc. - * @throws IOException - */ - @Test - public void testRpcTimeout() throws IOException { - Configuration localConfig = HBaseConfiguration.create(this.conf); - // This override mocks up our exists/get call to throw a RegionServerStoppedException. - localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); - int pause = 10; - localConfig.setInt("hbase.client.pause", pause); - localConfig.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10); - // Set the operation timeout to be < the pause. Expectation is that after first pause, we will - // fail out of the rpc because the rpc timeout will have been set to the operation tiemout - // and it has expired. Otherwise, if this functionality is broke, all retries will be run -- - // all ten of them -- and we'll get the RetriesExhaustedException exception. - localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1); - Connection connection = ConnectionFactory.createConnection(localConfig); - Table table = connection.getTable(TableName.META_TABLE_NAME); - Throwable t = null; - try { - // An exists call turns into a get w/ a flag. - table.exists(new Get(Bytes.toBytes("abc"))); - } catch (SocketTimeoutException e) { - // I expect this exception. - LOG.info("Got expected exception", e); - t = e; - } catch (RetriesExhaustedException e) { - // This is the old, unwanted behavior. If we get here FAIL!!! - fail(); - } finally { - table.close(); - connection.close(); - } - assertTrue(t != null); - } - - @Test - public void testDoNotRetryMetaTableAccessor() throws IOException { - this.conf.set("hbase.client.connection.impl", - RegionServerStoppedOnScannerOpenConnection.class.getName()); - try (Connection connection = ConnectionFactory.createConnection(conf)) { - MetaTableAccessor.fullScanRegions(connection); - } - } - - @Test - public void testDoNotRetryOnScanNext() throws IOException { - this.conf.set("hbase.client.connection.impl", - RegionServerStoppedOnScannerOpenConnection.class.getName()); - // Go against meta else we will try to find first region for the table on construction which - // means we'll have to do a bunch more mocking. Tests that go against meta only should be - // good for a bit of testing. - Connection connection = ConnectionFactory.createConnection(this.conf); - Table table = connection.getTable(TableName.META_TABLE_NAME); - ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY); - try { - Result result = null; - while ((result = scanner.next()) != null) { - LOG.info(Objects.toString(result)); - } - } finally { - scanner.close(); - table.close(); - connection.close(); - } - } - - @Test - public void testRegionServerStoppedOnScannerOpen() throws IOException { - this.conf.set("hbase.client.connection.impl", - RegionServerStoppedOnScannerOpenConnection.class.getName()); - // Go against meta else we will try to find first region for the table on construction which - // means we'll have to do a bunch more mocking. Tests that go against meta only should be - // good for a bit of testing. - Connection connection = ConnectionFactory.createConnection(conf); - Table table = connection.getTable(TableName.META_TABLE_NAME); - ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY); - try { - Result result = null; - while ((result = scanner.next()) != null) { - LOG.info(Objects.toString(result)); - } - } finally { - scanner.close(); - table.close(); - connection.close(); - } - } - - @Test - public void testConnectionClosedOnRegionLocate() throws IOException { - Configuration testConf = new Configuration(this.conf); - testConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); - // Go against meta else we will try to find first region for the table on construction which - // means we'll have to do a bunch more mocking. Tests that go against meta only should be - // good for a bit of testing. - Connection connection = ConnectionFactory.createConnection(testConf); - Table table = connection.getTable(TableName.META_TABLE_NAME); - connection.close(); - try { - Get get = new Get(Bytes.toBytes("dummyRow")); - table.get(get); - fail("Should have thrown DoNotRetryException but no exception thrown"); - } catch (Exception e) { - if (!(e instanceof DoNotRetryIOException)) { - String errMsg = - "Should have thrown DoNotRetryException but actually " + e.getClass().getSimpleName(); - LOG.error(errMsg, e); - fail(errMsg); - } - } finally { - table.close(); - } - } - - /** - * Override to shutdown going to zookeeper for cluster id and meta location. - */ - static class RegionServerStoppedOnScannerOpenConnection - extends ConnectionImplementation { - final ClientService.BlockingInterface stub; - - RegionServerStoppedOnScannerOpenConnection(Configuration conf, - ExecutorService pool, User user) throws IOException { - super(conf, pool, user); - // Mock up my stub so open scanner returns a scanner id and then on next, we throw - // exceptions for three times and then after that, we return no more to scan. - this.stub = Mockito.mock(ClientService.BlockingInterface.class); - long sid = 12345L; - try { - Mockito.when(stub.scan((RpcController)Mockito.any(), - (ClientProtos.ScanRequest)Mockito.any())). - thenReturn(ClientProtos.ScanResponse.newBuilder().setScannerId(sid).build()). - thenThrow(new ServiceException(new RegionServerStoppedException("From Mockito"))). - thenReturn(ClientProtos.ScanResponse.newBuilder().setScannerId(sid). - setMoreResults(false).build()); - } catch (ServiceException e) { - throw new IOException(e); - } - } - - @Override - public BlockingInterface getClient(ServerName sn) throws IOException { - return this.stub; - } - } - - /** - * Override to check we are setting rpc timeout right. - */ - static class RpcTimeoutConnection - extends ConnectionImplementation { - final ClientService.BlockingInterface stub; - - RpcTimeoutConnection(Configuration conf, ExecutorService pool, User user) - throws IOException { - super(conf, pool, user); - // Mock up my stub so an exists call -- which turns into a get -- throws an exception - this.stub = Mockito.mock(ClientService.BlockingInterface.class); - try { - Mockito.when(stub.get((RpcController)Mockito.any(), - (ClientProtos.GetRequest)Mockito.any())). - thenThrow(new ServiceException(new RegionServerStoppedException("From Mockito"))); - } catch (ServiceException e) { - throw new IOException(e); - } - } - - @Override - public BlockingInterface getClient(ServerName sn) throws IOException { - return this.stub; - } - } - - /** - * Fake many regionservers and many regions on a connection implementation. - */ - static class ManyServersManyRegionsConnection - extends ConnectionImplementation { - // All access should be synchronized - final Map serversByClient; - - /** - * Map of faked-up rows of a 'meta table'. - */ - final SortedMap> meta; - final AtomicLong sequenceids = new AtomicLong(0); - private final Configuration conf; - - ManyServersManyRegionsConnection(Configuration conf, - ExecutorService pool, User user) - throws IOException { - super(conf, pool, user); - int serverCount = conf.getInt("hbase.test.servers", 10); - this.serversByClient = new HashMap<>(serverCount); - this.meta = makeMeta(Bytes.toBytes( - conf.get("hbase.test.tablename", Bytes.toString(BIG_USER_TABLE))), - conf.getInt("hbase.test.regions", 100), - conf.getLong("hbase.test.namespace.span", 1000), - serverCount); - this.conf = conf; - } - - @Override - public ClientService.BlockingInterface getClient(ServerName sn) throws IOException { - // if (!sn.toString().startsWith("meta")) LOG.info(sn); - ClientService.BlockingInterface stub = null; - synchronized (this.serversByClient) { - stub = this.serversByClient.get(sn); - if (stub == null) { - stub = new FakeServer(this.conf, meta, sequenceids); - this.serversByClient.put(sn, stub); - } - } - return stub; - } - } - - static MultiResponse doMultiResponse(final SortedMap> meta, - final AtomicLong sequenceids, final MultiRequest request) { - // Make a response to match the request. Act like there were no failures. - ClientProtos.MultiResponse.Builder builder = ClientProtos.MultiResponse.newBuilder(); - // Per Region. - RegionActionResult.Builder regionActionResultBuilder = - RegionActionResult.newBuilder(); - ResultOrException.Builder roeBuilder = ResultOrException.newBuilder(); - for (RegionAction regionAction: request.getRegionActionList()) { - regionActionResultBuilder.clear(); - // Per Action in a Region. - for (ClientProtos.Action action: regionAction.getActionList()) { - roeBuilder.clear(); - // Return empty Result and proper index as result. - roeBuilder.setResult(ClientProtos.Result.getDefaultInstance()); - roeBuilder.setIndex(action.getIndex()); - regionActionResultBuilder.addResultOrException(roeBuilder.build()); - } - builder.addRegionActionResult(regionActionResultBuilder.build()); - } - return builder.build(); - } - - /** - * Fake 'server'. - * Implements the ClientService responding as though it were a 'server' (presumes a new - * ClientService.BlockingInterface made per server). - */ - static class FakeServer implements ClientService.BlockingInterface { - private AtomicInteger multiInvocationsCount = new AtomicInteger(0); - private final SortedMap> meta; - private final AtomicLong sequenceids; - private final long multiPause; - private final int tooManyMultiRequests; - - FakeServer(final Configuration c, final SortedMap> meta, - final AtomicLong sequenceids) { - this.meta = meta; - this.sequenceids = sequenceids; - - // Pause to simulate the server taking time applying the edits. This will drive up the - // number of threads used over in client. - this.multiPause = c.getLong("hbase.test.multi.pause.when.done", 0); - this.tooManyMultiRequests = c.getInt("hbase.test.multi.too.many", 3); - } - - @Override - public GetResponse get(RpcController controller, GetRequest request) - throws ServiceException { - boolean metaRegion = isMetaRegion(request.getRegion().getValue().toByteArray(), - request.getRegion().getType()); - if (!metaRegion) { - return doGetResponse(request); - } - return doMetaGetResponse(meta, request); - } - - private GetResponse doGetResponse(GetRequest request) { - ClientProtos.Result.Builder resultBuilder = ClientProtos.Result.newBuilder(); - ByteString row = request.getGet().getRow(); - resultBuilder.addCell(getStartCode(row)); - GetResponse.Builder builder = GetResponse.newBuilder(); - builder.setResult(resultBuilder.build()); - return builder.build(); - } - - @Override - public MutateResponse mutate(RpcController controller, - MutateRequest request) throws ServiceException { - throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); - } - - @Override - public ScanResponse scan(RpcController controller, - ScanRequest request) throws ServiceException { - // Presume it is a scan of meta for now. Not all scans provide a region spec expecting - // the server to keep reference by scannerid. TODO. - return doMetaScanResponse(meta, sequenceids, request); - } - - @Override - public BulkLoadHFileResponse bulkLoadHFile( - RpcController controller, BulkLoadHFileRequest request) - throws ServiceException { - throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); - } - - @Override - public CoprocessorServiceResponse execService( - RpcController controller, CoprocessorServiceRequest request) - throws ServiceException { - throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); - } - - @Override - public MultiResponse multi(RpcController controller, MultiRequest request) - throws ServiceException { - int concurrentInvocations = this.multiInvocationsCount.incrementAndGet(); - try { - if (concurrentInvocations >= tooManyMultiRequests) { - throw new ServiceException(new RegionTooBusyException("concurrentInvocations=" + - concurrentInvocations)); - } - Threads.sleep(multiPause); - return doMultiResponse(meta, sequenceids, request); - } finally { - this.multiInvocationsCount.decrementAndGet(); - } - } - - @Override - public CoprocessorServiceResponse execRegionServerService(RpcController controller, - CoprocessorServiceRequest request) throws ServiceException { - throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); - } - - @Override - public PrepareBulkLoadResponse prepareBulkLoad(RpcController controller, - PrepareBulkLoadRequest request) throws ServiceException { - throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); - } - - @Override - public CleanupBulkLoadResponse cleanupBulkLoad(RpcController controller, - CleanupBulkLoadRequest request) throws ServiceException { - throw new NotImplementedException(HConstants.NOT_IMPLEMENTED); - } - } - - static ScanResponse doMetaScanResponse(final SortedMap> meta, - final AtomicLong sequenceids, final ScanRequest request) { - ScanResponse.Builder builder = ScanResponse.newBuilder(); - int max = request.getNumberOfRows(); - int count = 0; - Map> tail = - request.hasScan()? meta.tailMap(request.getScan().getStartRow().toByteArray()): meta; - ClientProtos.Result.Builder resultBuilder = ClientProtos.Result.newBuilder(); - for (Map.Entry> e: tail.entrySet()) { - // Can be 0 on open of a scanner -- i.e. rpc to setup scannerid only. - if (max <= 0) break; - if (++count > max) break; - HRegionInfo hri = e.getValue().getFirst(); - ByteString row = UnsafeByteOperations.unsafeWrap(hri.getRegionName()); - resultBuilder.clear(); - resultBuilder.addCell(getRegionInfo(row, hri)); - resultBuilder.addCell(getServer(row, e.getValue().getSecond())); - resultBuilder.addCell(getStartCode(row)); - builder.addResults(resultBuilder.build()); - // Set more to false if we are on the last region in table. - if (hri.getEndKey().length <= 0) builder.setMoreResults(false); - else builder.setMoreResults(true); - } - // If no scannerid, set one. - builder.setScannerId(request.hasScannerId()? - request.getScannerId(): sequenceids.incrementAndGet()); - return builder.build(); - } - - static GetResponse doMetaGetResponse(final SortedMap> meta, - final GetRequest request) { - ClientProtos.Result.Builder resultBuilder = ClientProtos.Result.newBuilder(); - ByteString row = request.getGet().getRow(); - Pair p = meta.get(row.toByteArray()); - if (p != null) { - resultBuilder.addCell(getRegionInfo(row, p.getFirst())); - resultBuilder.addCell(getServer(row, p.getSecond())); - } - resultBuilder.addCell(getStartCode(row)); - GetResponse.Builder builder = GetResponse.newBuilder(); - builder.setResult(resultBuilder.build()); - return builder.build(); - } - - /** - * @param name region name or encoded region name. - * @param type - * @return True if we are dealing with a hbase:meta region. - */ - static boolean isMetaRegion(final byte [] name, final RegionSpecifierType type) { - switch (type) { - case REGION_NAME: - return Bytes.equals(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), name); - case ENCODED_REGION_NAME: - return Bytes.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), name); - default: throw new UnsupportedOperationException(); - } - } - - private final static ByteString CATALOG_FAMILY_BYTESTRING = - UnsafeByteOperations.unsafeWrap(HConstants.CATALOG_FAMILY); - private final static ByteString REGIONINFO_QUALIFIER_BYTESTRING = - UnsafeByteOperations.unsafeWrap(HConstants.REGIONINFO_QUALIFIER); - private final static ByteString SERVER_QUALIFIER_BYTESTRING = - UnsafeByteOperations.unsafeWrap(HConstants.SERVER_QUALIFIER); - - static CellProtos.Cell.Builder getBaseCellBuilder(final ByteString row) { - CellProtos.Cell.Builder cellBuilder = CellProtos.Cell.newBuilder(); - cellBuilder.setRow(row); - cellBuilder.setFamily(CATALOG_FAMILY_BYTESTRING); - cellBuilder.setTimestamp(System.currentTimeMillis()); - return cellBuilder; - } - - static CellProtos.Cell getRegionInfo(final ByteString row, final HRegionInfo hri) { - CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row); - cellBuilder.setQualifier(REGIONINFO_QUALIFIER_BYTESTRING); - cellBuilder.setValue(UnsafeByteOperations.unsafeWrap(hri.toByteArray())); - return cellBuilder.build(); - } - - static CellProtos.Cell getServer(final ByteString row, final ServerName sn) { - CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row); - cellBuilder.setQualifier(SERVER_QUALIFIER_BYTESTRING); - cellBuilder.setValue(ByteString.copyFromUtf8(sn.getHostAndPort())); - return cellBuilder.build(); - } - - static CellProtos.Cell getStartCode(final ByteString row) { - CellProtos.Cell.Builder cellBuilder = getBaseCellBuilder(row); - cellBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(HConstants.STARTCODE_QUALIFIER)); - // TODO: - cellBuilder.setValue(UnsafeByteOperations.unsafeWrap( - Bytes.toBytes(META_SERVERNAME.getStartcode()))); - return cellBuilder.build(); - } - - private static final byte [] BIG_USER_TABLE = Bytes.toBytes("t"); - - /** - * Format passed integer. Zero-pad. - * Copied from hbase-server PE class and small amendment. Make them share. - * @param number - * @return Returns zero-prefixed 10-byte wide decimal version of passed - * number (Does absolute in case number is negative). - */ - private static byte [] format(final long number) { - byte [] b = new byte[10]; - long d = number; - for (int i = b.length - 1; i >= 0; i--) { - b[i] = (byte)((d % 10) + '0'); - d /= 10; - } - return b; - } - - /** - * @param count - * @param namespaceSpan - * @return count regions - */ - private static HRegionInfo [] makeHRegionInfos(final byte [] tableName, final int count, - final long namespaceSpan) { - byte [] startKey = HConstants.EMPTY_BYTE_ARRAY; - byte [] endKey = HConstants.EMPTY_BYTE_ARRAY; - long interval = namespaceSpan / count; - HRegionInfo [] hris = new HRegionInfo[count]; - for (int i = 0; i < count; i++) { - if (i == 0) { - endKey = format(interval); - } else { - startKey = endKey; - if (i == count - 1) endKey = HConstants.EMPTY_BYTE_ARRAY; - else endKey = format((i + 1) * interval); - } - hris[i] = new HRegionInfo(TableName.valueOf(tableName), startKey, endKey); - } - return hris; - } - - /** - * @param count - * @return Return count servernames. - */ - private static ServerName [] makeServerNames(final int count) { - ServerName [] sns = new ServerName[count]; - for (int i = 0; i < count; i++) { - sns[i] = ServerName.valueOf("" + i + ".example.org", 16010, i); - } - return sns; - } - - /** - * Comparator for meta row keys. - */ - private static class MetaRowsComparator implements Comparator { - private final CellComparatorImpl delegate = CellComparatorImpl.META_COMPARATOR; - @Override - public int compare(byte[] left, byte[] right) { - return delegate.compareRows(new KeyValue.KeyOnlyKeyValue(left), right, 0, right.length); - } - } - - /** - * Create up a map that is keyed by meta row name and whose value is the HRegionInfo and - * ServerName to return for this row. - * @return Map with faked hbase:meta content in it. - */ - static SortedMap> makeMeta(final byte [] tableName, - final int regionCount, final long namespaceSpan, final int serverCount) { - // I need a comparator for meta rows so we sort properly. - SortedMap> meta = - new ConcurrentSkipListMap<>(new MetaRowsComparator()); - HRegionInfo [] hris = makeHRegionInfos(tableName, regionCount, namespaceSpan); - ServerName [] serverNames = makeServerNames(serverCount); - int per = regionCount / serverCount; - int count = 0; - for (HRegionInfo hri: hris) { - Pair p = new Pair<>(hri, serverNames[count++ / per]); - meta.put(hri.getRegionName(), p); - } - return meta; - } - - /** - * Code for each 'client' to run. - * - * @param id - * @param c - * @param sharedConnection - * @throws IOException - */ - static void cycle(int id, final Configuration c, final Connection sharedConnection) throws IOException { - long namespaceSpan = c.getLong("hbase.test.namespace.span", 1000000); - long startTime = System.currentTimeMillis(); - final int printInterval = 100000; - Random rd = new Random(id); - boolean get = c.getBoolean("hbase.test.do.gets", false); - TableName tableName = TableName.valueOf(BIG_USER_TABLE); - if (get) { - try (Table table = sharedConnection.getTable(tableName)){ - Stopwatch stopWatch = Stopwatch.createStarted(); - for (int i = 0; i < namespaceSpan; i++) { - byte [] b = format(rd.nextLong()); - Get g = new Get(b); - table.get(g); - if (i % printInterval == 0) { - LOG.info("Get " + printInterval + "/" + stopWatch.elapsed(java.util.concurrent.TimeUnit.MILLISECONDS)); - stopWatch.reset(); - stopWatch.start(); - } - } - LOG.info("Finished a cycle putting " + namespaceSpan + " in " + - (System.currentTimeMillis() - startTime) + "ms"); - } - } else { - try (BufferedMutator mutator = sharedConnection.getBufferedMutator(tableName)) { - Stopwatch stopWatch = Stopwatch.createStarted(); - for (int i = 0; i < namespaceSpan; i++) { - byte [] b = format(rd.nextLong()); - Put p = new Put(b); - p.addColumn(HConstants.CATALOG_FAMILY, b, b); - mutator.mutate(p); - if (i % printInterval == 0) { - LOG.info("Put " + printInterval + "/" + stopWatch.elapsed(java.util.concurrent.TimeUnit.MILLISECONDS)); - stopWatch.reset(); - stopWatch.start(); - } - } - LOG.info("Finished a cycle putting " + namespaceSpan + " in " + - (System.currentTimeMillis() - startTime) + "ms"); - } - } - } - - @Override - public int run(String[] arg0) throws Exception { - int errCode = 0; - // TODO: Make command options. - // How many servers to fake. - final int servers = 1; - // How many regions to put on the faked servers. - final int regions = 100000; - // How many 'keys' in the faked regions. - final long namespaceSpan = 50000000; - // How long to take to pause after doing a put; make this long if you want to fake a struggling - // server. - final long multiPause = 0; - // Check args make basic sense. - if ((namespaceSpan < regions) || (regions < servers)) { - throw new IllegalArgumentException("namespaceSpan=" + namespaceSpan + " must be > regions=" + - regions + " which must be > servers=" + servers); - } - - // Set my many servers and many regions faking connection in place. - getConf().set("hbase.client.connection.impl", - ManyServersManyRegionsConnection.class.getName()); - // Use simple kv registry rather than zk - getConf().set("hbase.client.registry.impl", SimpleRegistry.class.getName()); - // When to report fails. Default is we report the 10th. This means we'll see log everytime - // an exception is thrown -- usually RegionTooBusyException when we have more than - // hbase.test.multi.too.many requests outstanding at any time. - getConf().setInt("hbase.client.start.log.errors.counter", 0); - - // Ugly but this is only way to pass in configs.into ManyServersManyRegionsConnection class. - getConf().setInt("hbase.test.regions", regions); - getConf().setLong("hbase.test.namespace.span", namespaceSpan); - getConf().setLong("hbase.test.servers", servers); - getConf().set("hbase.test.tablename", Bytes.toString(BIG_USER_TABLE)); - getConf().setLong("hbase.test.multi.pause.when.done", multiPause); - // Let there be ten outstanding requests at a time before we throw RegionBusyException. - getConf().setInt("hbase.test.multi.too.many", 10); - final int clients = 2; - - // Have them all share the same connection so they all share the same instance of - // ManyServersManyRegionsConnection so I can keep an eye on how many requests by server. - final ExecutorService pool = Executors.newCachedThreadPool(Threads.getNamedThreadFactory("p")); - // Executors.newFixedThreadPool(servers * 10, Threads.getNamedThreadFactory("p")); - // Share a connection so I can keep counts in the 'server' on concurrency. - final Connection sharedConnection = ConnectionFactory.createConnection(getConf()/*, pool*/); - try { - Thread [] ts = new Thread[clients]; - for (int j = 0; j < ts.length; j++) { - final int id = j; - ts[j] = new Thread("" + j) { - final Configuration c = getConf(); - - @Override - public void run() { - try { - cycle(id, c, sharedConnection); - } catch (IOException e) { - e.printStackTrace(); - } - } - }; - ts[j].start(); - } - for (int j = 0; j < ts.length; j++) { - ts[j].join(); - } - } finally { - sharedConnection.close(); - } - return errCode; - } - - /** - * Run a client instance against a faked up server. - * @param args TODO - * @throws Exception - */ - public static void main(String[] args) throws Exception { - System.exit(ToolRunner.run(HBaseConfiguration.create(), new TestClientNoCluster(), args)); - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java deleted file mode 100644 index 9f1f6f3251af..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java +++ /dev/null @@ -1,523 +0,0 @@ -/** - * 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 static org.hamcrest.CoreMatchers.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.Iterator; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScanner; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.Type; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.mockito.InOrder; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -/** - * Test the ClientScanner. - */ -@Category(SmallTests.class) -public class TestClientScanner { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestClientScanner.class); - - Scan scan; - ExecutorService pool; - Configuration conf; - - ConnectionImplementation clusterConn; - RpcRetryingCallerFactory rpcFactory; - RpcControllerFactory controllerFactory; - - @Rule - public TestName name = new TestName(); - - @Before - public void setup() throws IOException { - clusterConn = Mockito.mock(ConnectionImplementation.class); - rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class); - controllerFactory = Mockito.mock(RpcControllerFactory.class); - pool = Executors.newSingleThreadExecutor(); - scan = new Scan(); - conf = new Configuration(); - Mockito.when(clusterConn.getConfiguration()).thenReturn(conf); - } - - @After - public void teardown() { - if (null != pool) { - pool.shutdownNow(); - } - } - - private static class MockClientScanner extends ClientSimpleScanner { - - private boolean rpcFinished = false; - private boolean rpcFinishedFired = false; - private boolean initialized = false; - - public MockClientScanner(final Configuration conf, final Scan scan, final TableName tableName, - ConnectionImplementation connection, RpcRetryingCallerFactory rpcFactory, - RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout) - throws IOException { - super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - primaryOperationTimeout); - } - - @Override - protected boolean moveToNextRegion() { - if (!initialized) { - initialized = true; - return super.moveToNextRegion(); - } - if (!rpcFinished) { - return super.moveToNextRegion(); - } - // Enforce that we don't short-circuit more than once - if (rpcFinishedFired) { - throw new RuntimeException("Expected nextScanner to only be called once after " + - " short-circuit was triggered."); - } - rpcFinishedFired = true; - return false; - } - - public void setRpcFinished(boolean rpcFinished) { - this.rpcFinished = rpcFinished; - } - } - - @Test - @SuppressWarnings("unchecked") - public void testNoResultsHint() throws IOException { - final Result[] results = new Result[1]; - KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - results[0] = Result.create(new Cell[] {kv1}); - - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(), - Mockito.anyInt())).thenAnswer(new Answer() { - private int count = 0; - @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { - ScannerCallableWithReplicas callable = invocation.getArgument(0); - switch (count) { - case 0: // initialize - count++; - callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.UNKNOWN); - return results; - case 1: // detect no more results - case 2: // close - count++; - return new Result[0]; - default: - throw new RuntimeException("Expected only 2 invocations"); - } - } - }); - - // Set a much larger cache and buffer size than we'll provide - scan.setCaching(100); - scan.setMaxResultSize(1000*1000); - - try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf(name.getMethodName()), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - - scanner.setRpcFinished(true); - - InOrder inOrder = Mockito.inOrder(caller); - - scanner.loadCache(); - - // One for fetching the results - // One for fetching empty results and quit as we do not have moreResults hint. - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(), Mockito.anyInt()); - - assertEquals(1, scanner.cache.size()); - Result r = scanner.cache.poll(); - assertNotNull(r); - CellScanner cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv1, cs.current()); - assertFalse(cs.advance()); - } - } - - @Test - @SuppressWarnings("unchecked") - public void testSizeLimit() throws IOException { - final Result[] results = new Result[1]; - KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - results[0] = Result.create(new Cell[] {kv1}); - - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(), - Mockito.anyInt())).thenAnswer(new Answer() { - private int count = 0; - @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { - ScannerCallableWithReplicas callable = invocation.getArgument(0); - switch (count) { - case 0: // initialize - count++; - // if we set no here the implementation will trigger a close - callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES); - return results; - case 1: // close - count++; - return null; - default: - throw new RuntimeException("Expected only 2 invocations"); - } - } - }); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - - // Set a much larger cache - scan.setCaching(100); - // The single key-value will exit the loop - scan.setMaxResultSize(1); - - try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf(name.getMethodName()), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - InOrder inOrder = Mockito.inOrder(caller); - - scanner.loadCache(); - - inOrder.verify(caller, Mockito.times(1)).callWithoutRetries( - Mockito.any(), Mockito.anyInt()); - - assertEquals(1, scanner.cache.size()); - Result r = scanner.cache.poll(); - assertNotNull(r); - CellScanner cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv1, cs.current()); - assertFalse(cs.advance()); - } - } - - @Test - @SuppressWarnings("unchecked") - public void testCacheLimit() throws IOException { - KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - KeyValue kv3 = new KeyValue(Bytes.toBytes("row3"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - final Result[] results = new Result[] {Result.create(new Cell[] {kv1}), - Result.create(new Cell[] {kv2}), Result.create(new Cell[] {kv3})}; - - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(), - Mockito.anyInt())).thenAnswer(new Answer() { - private int count = 0; - @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { - ScannerCallableWithReplicas callable = invocation.getArgument(0); - switch (count) { - case 0: // initialize - count++; - // if we set no here the implementation will trigger a close - callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES); - return results; - case 1: // close - count++; - return null; - default: - throw new RuntimeException("Expected only 2 invocations"); - } - } - }); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - - // Set a small cache - scan.setCaching(1); - // Set a very large size - scan.setMaxResultSize(1000*1000); - - try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf(name.getMethodName()), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - InOrder inOrder = Mockito.inOrder(caller); - - scanner.loadCache(); - - inOrder.verify(caller, Mockito.times(1)).callWithoutRetries( - Mockito.any(), Mockito.anyInt()); - - assertEquals(3, scanner.cache.size()); - Result r = scanner.cache.poll(); - assertNotNull(r); - CellScanner cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv1, cs.current()); - assertFalse(cs.advance()); - - r = scanner.cache.poll(); - assertNotNull(r); - cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv2, cs.current()); - assertFalse(cs.advance()); - - r = scanner.cache.poll(); - assertNotNull(r); - cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv3, cs.current()); - assertFalse(cs.advance()); - } - } - - @Test - @SuppressWarnings("unchecked") - public void testNoMoreResults() throws IOException { - final Result[] results = new Result[1]; - KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - results[0] = Result.create(new Cell[] {kv1}); - - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(), - Mockito.anyInt())).thenAnswer(new Answer() { - private int count = 0; - @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { - ScannerCallableWithReplicas callable = invocation.getArgument(0); - switch (count) { - case 0: // initialize - count++; - callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO); - return results; - case 1: // close - count++; - return null; - default: - throw new RuntimeException("Expected only 2 invocations"); - } - } - }); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - - // Set a much larger cache and buffer size than we'll provide - scan.setCaching(100); - scan.setMaxResultSize(1000*1000); - - try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf(name.getMethodName()), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - scanner.setRpcFinished(true); - - InOrder inOrder = Mockito.inOrder(caller); - - scanner.loadCache(); - - inOrder.verify(caller, Mockito.times(1)).callWithoutRetries( - Mockito.any(), Mockito.anyInt()); - - assertEquals(1, scanner.cache.size()); - Result r = scanner.cache.poll(); - assertNotNull(r); - CellScanner cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv1, cs.current()); - assertFalse(cs.advance()); - } - } - - @Test - @SuppressWarnings("unchecked") - public void testMoreResults() throws IOException { - final Result[] results1 = new Result[1]; - KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - results1[0] = Result.create(new Cell[] {kv1}); - - final Result[] results2 = new Result[1]; - KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1, - Type.Maximum); - results2[0] = Result.create(new Cell[] {kv2}); - - - RpcRetryingCaller caller = Mockito.mock(RpcRetryingCaller.class); - - Mockito.when(rpcFactory. newCaller()).thenReturn(caller); - Mockito.when(caller.callWithoutRetries(Mockito.any(), - Mockito.anyInt())).thenAnswer(new Answer() { - private int count = 0; - @Override - public Result[] answer(InvocationOnMock invocation) throws Throwable { - ScannerCallableWithReplicas callable = invocation.getArgument(0); - switch (count) { - case 0: // initialize - count++; - callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.YES); - return results1; - case 1: - count++; - // The server reports back false WRT more results - callable.currentScannerCallable.setMoreResultsInRegion(MoreResults.NO); - return results2; - case 2: // close - count++; - return null; - default: - throw new RuntimeException("Expected only 3 invocations"); - } - } - }); - - // Set a much larger cache and buffer size than we'll provide - scan.setCaching(100); - scan.setMaxResultSize(1000*1000); - - try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf(name.getMethodName()), - clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) { - InOrder inOrder = Mockito.inOrder(caller); - scanner.setRpcFinished(true); - - scanner.loadCache(); - - inOrder.verify(caller, Mockito.times(2)).callWithoutRetries( - Mockito.any(), Mockito.anyInt()); - - assertEquals(2, scanner.cache.size()); - Result r = scanner.cache.poll(); - assertNotNull(r); - CellScanner cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv1, cs.current()); - assertFalse(cs.advance()); - - r = scanner.cache.poll(); - assertNotNull(r); - cs = r.cellScanner(); - assertTrue(cs.advance()); - assertEquals(kv2, cs.current()); - assertFalse(cs.advance()); - } - } - - /** - * Tests the case where all replicas of a region throw an exception. It should not cause a hang - * but the exception should propagate to the client - */ - @Test - public void testExceptionsFromReplicasArePropagated() throws IOException { - scan.setConsistency(Consistency.TIMELINE); - - // Mock a caller which calls the callable for ScannerCallableWithReplicas, - // but throws an exception for the actual scanner calls via callWithRetries. - rpcFactory = new MockRpcRetryingCallerFactory(conf); - conf.set(RpcRetryingCallerFactory.CUSTOM_CALLER_CONF_KEY, - MockRpcRetryingCallerFactory.class.getName()); - - // mock 3 replica locations - when(clusterConn.locateRegion((TableName)any(), (byte[])any(), anyBoolean(), - anyBoolean(), anyInt())).thenReturn(new RegionLocations(null, null, null)); - - try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf(name.getMethodName()), - clusterConn, rpcFactory, new RpcControllerFactory(conf), pool, Integer.MAX_VALUE)) { - Iterator iter = scanner.iterator(); - while (iter.hasNext()) { - iter.next(); - } - fail("Should have failed with RetriesExhaustedException"); - } catch (RuntimeException expected) { - assertThat(expected.getCause(), instanceOf(RetriesExhaustedException.class)); - } - } - - public static class MockRpcRetryingCallerFactory extends RpcRetryingCallerFactory { - - public MockRpcRetryingCallerFactory(Configuration conf) { - super(conf); - } - - @Override - public RpcRetryingCaller newCaller() { - return new RpcRetryingCaller() { - @Override - public void cancel() { - } - @Override - public T callWithRetries(RetryingCallable callable, int callTimeout) - throws IOException, RuntimeException { - throw new IOException("Scanner exception"); - } - - @Override - public T callWithoutRetries(RetryingCallable callable, int callTimeout) - throws IOException, RuntimeException { - try { - return callable.call(callTimeout); - } catch (IOException e) { - throw e; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - }; - } - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java deleted file mode 100644 index 22e718bf95f5..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestDelayingRunner.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * 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 static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(SmallTests.class) -public class TestDelayingRunner { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestDelayingRunner.class); - - private static final TableName DUMMY_TABLE = - TableName.valueOf("DUMMY_TABLE"); - private static final byte[] DUMMY_BYTES_1 = Bytes.toBytes("DUMMY_BYTES_1"); - private static final byte[] DUMMY_BYTES_2 = Bytes.toBytes("DUMMY_BYTES_2"); - private static HRegionInfo hri1 = - new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1); - - @SuppressWarnings({ "rawtypes", "unchecked" }) - @Test - public void testDelayingRunner() throws Exception{ - MultiAction ma = new MultiAction(); - ma.add(hri1.getRegionName(), new Action(new Put(DUMMY_BYTES_1), 0)); - final AtomicLong endTime = new AtomicLong(); - final long sleepTime = 1000; - DelayingRunner runner = new DelayingRunner(sleepTime, ma.actions.entrySet().iterator().next()); - runner.setRunner(new Runnable() { - @Override - public void run() { - endTime.set(EnvironmentEdgeManager.currentTime()); - } - }); - long startTime = EnvironmentEdgeManager.currentTime(); - runner.run(); - long delay = endTime.get() - startTime; - assertTrue("DelayingRunner did not delay long enough", delay >= sleepTime); - assertFalse("DelayingRunner delayed too long", delay > sleepTime + sleepTime*0.2); - } - -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java deleted file mode 100644 index 01740e98461f..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java +++ /dev/null @@ -1,188 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; - -@Category({ClientTests.class, SmallTests.class}) -public class TestProcedureFuture { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestProcedureFuture.class); - - private static class TestFuture extends HBaseAdmin.ProcedureFuture { - private boolean postOperationResultCalled = false; - private boolean waitOperationResultCalled = false; - private boolean getProcedureResultCalled = false; - private boolean convertResultCalled = false; - - public TestFuture(final HBaseAdmin admin, final Long procId) { - super(admin, procId); - } - - public boolean wasPostOperationResultCalled() { - return postOperationResultCalled; - } - - public boolean wasWaitOperationResultCalled() { - return waitOperationResultCalled; - } - - public boolean wasGetProcedureResultCalled() { - return getProcedureResultCalled; - } - - public boolean wasConvertResultCalled() { - return convertResultCalled; - } - - @Override - protected GetProcedureResultResponse getProcedureResult( - final GetProcedureResultRequest request) throws IOException { - getProcedureResultCalled = true; - return GetProcedureResultResponse.newBuilder() - .setState(GetProcedureResultResponse.State.FINISHED) - .build(); - } - - @Override - protected Void convertResult(final GetProcedureResultResponse response) throws IOException { - convertResultCalled = true; - return null; - } - - @Override - protected Void waitOperationResult(final long deadlineTs) - throws IOException, TimeoutException { - waitOperationResultCalled = true; - return null; - } - - @Override - protected Void postOperationResult(final Void result, final long deadlineTs) - throws IOException, TimeoutException { - postOperationResultCalled = true; - return result; - } - } - - /** - * When a master return a result with procId, - * we are skipping the waitOperationResult() call, - * since we are getting the procedure result. - */ - @Test - public void testWithProcId() throws Exception { - HBaseAdmin admin = Mockito.mock(HBaseAdmin.class); - TestFuture f = new TestFuture(admin, 100L); - f.get(1, TimeUnit.MINUTES); - - assertTrue("expected getProcedureResult() to be called", f.wasGetProcedureResultCalled()); - assertTrue("expected convertResult() to be called", f.wasConvertResultCalled()); - assertFalse("unexpected waitOperationResult() called", f.wasWaitOperationResultCalled()); - assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled()); - } - - /** - * Verify that the spin loop for the procedure running works. - */ - @Test - public void testWithProcIdAndSpinning() throws Exception { - final AtomicInteger spinCount = new AtomicInteger(0); - HBaseAdmin admin = Mockito.mock(HBaseAdmin.class); - TestFuture f = new TestFuture(admin, 100L) { - @Override - protected GetProcedureResultResponse getProcedureResult( - final GetProcedureResultRequest request) throws IOException { - boolean done = spinCount.incrementAndGet() >= 10; - return GetProcedureResultResponse.newBuilder() - .setState(done ? GetProcedureResultResponse.State.FINISHED : - GetProcedureResultResponse.State.RUNNING) - .build(); - } - }; - f.get(1, TimeUnit.MINUTES); - - assertEquals(10, spinCount.get()); - assertTrue("expected convertResult() to be called", f.wasConvertResultCalled()); - assertFalse("unexpected waitOperationResult() called", f.wasWaitOperationResultCalled()); - assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled()); - } - - /** - * When a master return a result without procId, - * we are skipping the getProcedureResult() call. - */ - @Test - public void testWithoutProcId() throws Exception { - HBaseAdmin admin = Mockito.mock(HBaseAdmin.class); - TestFuture f = new TestFuture(admin, null); - f.get(1, TimeUnit.MINUTES); - - assertFalse("unexpected getProcedureResult() called", f.wasGetProcedureResultCalled()); - assertFalse("unexpected convertResult() called", f.wasConvertResultCalled()); - assertTrue("expected waitOperationResult() to be called", f.wasWaitOperationResultCalled()); - assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled()); - } - - /** - * When a new client with procedure support tries to ask an old-master without proc-support - * the procedure result we get a DoNotRetryIOException (which is an UnsupportedOperationException) - * The future should trap that and fallback to the waitOperationResult(). - * - * This happens when the operation calls happens on a "new master" but while we are waiting - * the operation to be completed, we failover on an "old master". - */ - @Test - public void testOnServerWithNoProcedureSupport() throws Exception { - HBaseAdmin admin = Mockito.mock(HBaseAdmin.class); - TestFuture f = new TestFuture(admin, 100L) { - @Override - protected GetProcedureResultResponse getProcedureResult( - final GetProcedureResultRequest request) throws IOException { - super.getProcedureResult(request); - throw new DoNotRetryIOException(new UnsupportedOperationException("getProcedureResult")); - } - }; - f.get(1, TimeUnit.MINUTES); - - assertTrue("expected getProcedureResult() to be called", f.wasGetProcedureResultCalled()); - assertFalse("unexpected convertResult() called", f.wasConvertResultCalled()); - assertTrue("expected waitOperationResult() to be called", f.wasWaitOperationResultCalled()); - assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled()); - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java deleted file mode 100644 index 7eb2b94f079c..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java +++ /dev/null @@ -1,99 +0,0 @@ -/** - * 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.HBaseClassTestRule; -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.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.runners.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -@Category({ ClientTests.class, SmallTests.class }) -public class TestReversedScannerCallable { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestReversedScannerCallable.class); - - @Mock - private ConnectionImplementation connection; - @Mock - private Scan scan; - @Mock - private RpcControllerFactory rpcFactory; - @Mock - private RegionLocations regionLocations; - - private final byte[] ROW = Bytes.toBytes("row1"); - - @Before - public void setUp() throws Exception { - byte[] ROW_BEFORE = ConnectionUtils.createCloseRowBefore(ROW); - - Configuration conf = Mockito.mock(Configuration.class); - HRegionLocation regionLocation = Mockito.mock(HRegionLocation.class); - ServerName serverName = Mockito.mock(ServerName.class); - - Mockito.when(connection.getConfiguration()).thenReturn(conf); - Mockito.when(regionLocations.size()).thenReturn(1); - Mockito.when(regionLocations.getRegionLocation(0)).thenReturn(regionLocation); - Mockito.when(regionLocation.getHostname()).thenReturn("localhost"); - Mockito.when(regionLocation.getServerName()).thenReturn(serverName); - Mockito.when(scan.includeStartRow()).thenReturn(true); - Mockito.when(scan.getStartRow()).thenReturn(ROW); - } - - @Test - public void testPrepareDoesNotUseCache() throws Exception { - TableName tableName = TableName.valueOf("MyTable"); - Mockito.when(connection.relocateRegion(tableName, ROW, 0)).thenReturn(regionLocations); - - ReversedScannerCallable callable = - new ReversedScannerCallable(connection, tableName, scan, null, rpcFactory); - callable.prepare(true); - - Mockito.verify(connection).relocateRegion(tableName, ROW, 0); - } - - @Test - public void testPrepareUsesCache() throws Exception { - TableName tableName = TableName.valueOf("MyTable"); - Mockito.when(connection.locateRegion(tableName, ROW, true, true, 0)) - .thenReturn(regionLocations); - - ReversedScannerCallable callable = - new ReversedScannerCallable(connection, tableName, scan, null, rpcFactory); - callable.prepare(false); - - Mockito.verify(connection).locateRegion(tableName, ROW, true, true, 0); - } -} diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java deleted file mode 100644 index dba1a369522f..000000000000 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java +++ /dev/null @@ -1,198 +0,0 @@ -/** - * 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 static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; - -/** - * Test snapshot logic from the client - */ -@Category({SmallTests.class, ClientTests.class}) -public class TestSnapshotFromAdmin { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestSnapshotFromAdmin.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFromAdmin.class); - - @Rule - public TestName name = new TestName(); - - /** - * Test that the logic for doing 'correct' back-off based on exponential increase and the max-time - * passed from the server ensures the correct overall waiting for the snapshot to finish. - * @throws Exception - */ - @Test - public void testBackoffLogic() throws Exception { - final int pauseTime = 100; - final int maxWaitTime = - HConstants.RETRY_BACKOFF[HConstants.RETRY_BACKOFF.length - 1] * pauseTime; - final int numRetries = HConstants.RETRY_BACKOFF.length; - // calculate the wait time, if we just do straight backoff (ignoring the expected time from - // master) - long ignoreExpectedTime = 0; - for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) { - ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime; - } - // the correct wait time, capping at the maxTime/tries + fudge room - final long time = pauseTime * 3L + ((maxWaitTime / numRetries) * 3) + 300L; - assertTrue("Capped snapshot wait time isn't less that the uncapped backoff time " - + "- further testing won't prove anything.", time < ignoreExpectedTime); - - // setup the mocks - ConnectionImplementation mockConnection = Mockito - .mock(ConnectionImplementation.class); - Configuration conf = HBaseConfiguration.create(); - // setup the conf to match the expected properties - conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries); - conf.setLong("hbase.client.pause", pauseTime); - - // mock the master admin to our mock - MasterKeepAliveConnection mockMaster = Mockito.mock(MasterKeepAliveConnection.class); - Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); - Mockito.when(mockConnection.getMaster()).thenReturn(mockMaster); - // we need a real retrying caller - RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); - RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); - Mockito.when(controllerFactory.newController()).thenReturn( - Mockito.mock(HBaseRpcController.class)); - Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); - Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); - // set the max wait time for the snapshot to complete - SnapshotResponse response = SnapshotResponse.newBuilder() - .setExpectedTimeout(maxWaitTime) - .build(); - Mockito - .when( - mockMaster.snapshot((RpcController) Mockito.any(), - Mockito.any())).thenReturn(response); - // setup the response - IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder(); - builder.setDone(false); - // first five times, we return false, last we get success - Mockito.when( - mockMaster.isSnapshotDone((RpcController) Mockito.any(), - Mockito.any())).thenReturn(builder.build(), builder.build(), - builder.build(), builder.build(), builder.build(), builder.setDone(true).build()); - - // setup the admin and run the test - Admin admin = new HBaseAdmin(mockConnection); - String snapshot = "snapshot"; - final TableName table = TableName.valueOf(name.getMethodName()); - // get start time - long start = System.currentTimeMillis(); - admin.snapshot(snapshot, table); - long finish = System.currentTimeMillis(); - long elapsed = (finish - start); - assertTrue("Elapsed time:" + elapsed + " is more than expected max:" + time, elapsed <= time); - admin.close(); - } - - /** - * Make sure that we validate the snapshot name and the table name before we pass anything across - * the wire - * @throws Exception on failure - */ - @Test - public void testValidateSnapshotName() throws Exception { - ConnectionImplementation mockConnection = Mockito - .mock(ConnectionImplementation.class); - Configuration conf = HBaseConfiguration.create(); - Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); - // we need a real retrying caller - RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); - RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); - Mockito.when(controllerFactory.newController()).thenReturn( - Mockito.mock(HBaseRpcController.class)); - Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); - Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); - Admin admin = new HBaseAdmin(mockConnection); - // check that invalid snapshot names fail - failSnapshotStart(admin, new SnapshotDescription(HConstants.SNAPSHOT_DIR_NAME)); - failSnapshotStart(admin, new SnapshotDescription("-snapshot")); - failSnapshotStart(admin, new SnapshotDescription("snapshot fails")); - failSnapshotStart(admin, new SnapshotDescription("snap$hot")); - failSnapshotStart(admin, new SnapshotDescription("snap:hot")); - // check the table name also get verified - failSnapshotDescriptorCreation("snapshot", ".table"); - failSnapshotDescriptorCreation("snapshot", "-table"); - failSnapshotDescriptorCreation("snapshot", "table fails"); - failSnapshotDescriptorCreation("snapshot", "tab%le"); - - // mock the master connection - MasterKeepAliveConnection master = Mockito.mock(MasterKeepAliveConnection.class); - Mockito.when(mockConnection.getMaster()).thenReturn(master); - SnapshotResponse response = SnapshotResponse.newBuilder().setExpectedTimeout(0).build(); - Mockito.when( - master.snapshot((RpcController) Mockito.any(), Mockito.any())) - .thenReturn(response); - IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build(); - Mockito.when( - master.isSnapshotDone((RpcController) Mockito.any(), - Mockito.any())).thenReturn(doneResponse); - - // make sure that we can use valid names - admin.snapshot(new SnapshotDescription("snapshot", TableName.valueOf(name.getMethodName()))); - } - - private void failSnapshotStart(Admin admin, SnapshotDescription snapshot) - throws IOException { - try { - admin.snapshot(snapshot); - fail("Snapshot should not have succeed with name:" + snapshot.getName()); - } catch (IllegalArgumentException e) { - LOG.debug("Correctly failed to start snapshot:" + e.getMessage()); - } - } - - private void failSnapshotDescriptorCreation(final String snapshotName, final String tableName) { - try { - new SnapshotDescription(snapshotName, tableName); - fail("SnapshotDescription should not have succeed with name:" + snapshotName); - } catch (IllegalArgumentException e) { - LOG.debug("Correctly failed to create SnapshotDescription:" + e.getMessage()); - } - } -} diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index 1e0045025519..dc06b0a57558 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -1981,8 +1980,6 @@ private static void setJobConf(Job job, int numMappers, long numNodes, } public static void setJobScannerConf(Job job) { - // Make sure scanners log something useful to make debugging possible. - job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true); job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java index ffad7d3a20d4..aedc676edfe7 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java @@ -20,12 +20,19 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.BufferedReader; +import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.InterruptedIOException; import java.util.Random; import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -39,25 +46,28 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestingUtility; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; -import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutatorParams; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.NMapInputFormat; +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapper; import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl; +import org.apache.hadoop.hbase.mapreduce.WALPlayer; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; @@ -66,24 +76,13 @@ import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.hbase.wal.WALEdit; -import org.apache.hadoop.hbase.wal.WALKey; -import org.apache.hadoop.hbase.mapreduce.WALPlayer; -import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; -import java.io.BufferedReader; -import java.io.FileNotFoundException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.InterruptedIOException; -import java.util.SortedSet; -import java.util.TreeSet; -import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.util.ToolRunner; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; @@ -519,8 +518,6 @@ private int doSearch(Configuration conf, String keysDir) throws Exception { } private static void setJobScannerConf(Job job) { - // Make sure scanners log something useful to make debugging possible. - job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true); long lpr = job.getConfiguration().getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT) / 100; job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, (int)lpr); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java index f524de4e1320..b15cee6aa652 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapper; @@ -339,7 +337,6 @@ private Job doVerify(Configuration conf, HTableDescriptor htd, String... auths) } private static void setJobScannerConf(Job job) { - job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true); long lpr = job.getConfiguration().getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT) / 100; job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, (int) lpr); } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java index a49d0ec5c3ea..af63a3800bed 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java @@ -18,24 +18,24 @@ */ package org.apache.hadoop.hbase.mapred; +import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT; + import java.io.IOException; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.client.ConnectionConfiguration; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableInputFormat; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; - -import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Iterate over an HBase table data, return (Text, RowResult) pairs @@ -116,7 +116,7 @@ byte[] getStartRow() { public void setHTable(Table htable) { Configuration conf = htable.getConfiguration(); logScannerActivity = conf.getBoolean( - ScannerCallable.LOG_SCANNER_ACTIVITY, false); + ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false); logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100); this.htable = htable; } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java index 28f4da127da0..d7a2ccb1d966 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java @@ -20,23 +20,23 @@ import java.io.IOException; import java.lang.reflect.Method; import java.util.Map; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.client.ConnectionConfiguration; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; -import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -122,7 +122,7 @@ protected static Method retrieveGetCounterWithStringsParams(TaskAttemptContext c public void setHTable(Table htable) { Configuration conf = htable.getConfiguration(); logScannerActivity = conf.getBoolean( - ScannerCallable.LOG_SCANNER_ACTIVITY, false); + ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false); logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100); this.htable = htable; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java index c1b84891a031..42e78bbd007d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java @@ -19,27 +19,25 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The cleaner to delete the expired MOB files. @@ -95,7 +93,6 @@ public int run(String[] args) throws Exception { String tableName = args[0]; String familyName = args[1]; TableName tn = TableName.valueOf(tableName); - HBaseAdmin.available(getConf()); Connection connection = ConnectionFactory.createConnection(getConf()); Admin admin = connection.getAdmin(); try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index b67751674e1d..90e425e28463 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; -import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.locking.EntityLock; import org.apache.hadoop.hbase.client.locking.LockServiceClient; @@ -351,7 +350,6 @@ public class HRegionServer extends HasThread implements // RPC client. Used to make the stub above that does region server status checking. RpcClient rpcClient; - private RpcRetryingCallerFactory rpcRetryingCallerFactory; private RpcControllerFactory rpcControllerFactory; private UncaughtExceptionHandler uncaughtExceptionHandler; @@ -586,7 +584,6 @@ public HRegionServer(Configuration conf) throws IOException { serverName = ServerName.valueOf(hostName, this.rpcServices.isa.getPort(), this.startcode); rpcControllerFactory = RpcControllerFactory.instantiate(this.conf); - rpcRetryingCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf); // login the zookeeper client principal (if using security) ZKUtil.loginClient(this.conf, HConstants.ZK_CLIENT_KEYTAB_FILE, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java index 67e945f245f7..3c86149a7c0f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.replication.TableCFs; import org.apache.hadoop.hbase.io.WALLink; import org.apache.hadoop.hbase.procedure2.util.StringUtils; @@ -204,9 +203,7 @@ protected static void printUsageAndExit(final String message, final int exitCode } private int dumpReplicationQueues(DumpOptions opts) throws Exception { - Configuration conf = getConf(); - HBaseAdmin.available(conf); Connection connection = ConnectionFactory.createConnection(conf); Admin admin = connection.getAdmin(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index 2c129a8aa7da..a16def9a593e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -17,30 +17,15 @@ */ package org.apache.hadoop.hbase.client; -import java.io.IOException; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; -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.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.util.Threads; import org.mockito.Mockito; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; - /** * {@link ConnectionImplementation} testing utility. */ public class HConnectionTestingUtility { + /* * Not part of {@link HBaseTestingUtility} because this class is not in same package as {@link * ConnectionImplementation}. Would have to reveal ugly {@link ConnectionImplementation} innards @@ -56,137 +41,10 @@ public class HConnectionTestingUtility { * @return ConnectionImplementation object for conf * @throws ZooKeeperConnectionException */ - public static ConnectionImplementation getMockedConnection(final Configuration conf) + public static Connection getMockedConnection(final Configuration conf) throws ZooKeeperConnectionException { - ConnectionImplementation connection = Mockito.mock(ConnectionImplementation.class); + Connection connection = Mockito.mock(Connection.class); Mockito.when(connection.getConfiguration()).thenReturn(conf); - Mockito.when(connection.getRpcControllerFactory()) - .thenReturn(Mockito.mock(RpcControllerFactory.class)); - // we need a real retrying caller - RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); - Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); return connection; } - - /** - * Calls {@link #getMockedConnection(Configuration)} and then mocks a few more of the popular - * {@link ConnectionImplementation} methods so they do 'normal' operation (see return doc below - * for list). Be sure to shutdown the connection when done by calling {@link Connection#close()} - * else it will stick around; this is probably not what you want. - * @param conf Configuration to use - * @param admin An AdminProtocol; can be null but is usually itself a mock. - * @param client A ClientProtocol; can be null but is usually itself a mock. - * @param sn ServerName to include in the region location returned by this connection - * @param hri RegionInfo to include in the location returned when getRegionLocator is called on - * the mocked connection - * @return Mock up a connection that returns a {@link Configuration} when - * {@link ConnectionImplementation#getConfiguration()} is called, a 'location' when - * {@link ConnectionImplementation#getRegionLocation(TableName, byte[], boolean)} is - * called, and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} - * instance when {@link ConnectionImplementation#getAdmin(ServerName)} is called, returns - * the passed {@link ClientProtos.ClientService.BlockingInterface} instance when - * {@link ConnectionImplementation#getClient(ServerName)} is called (Be sure to call - * {@link Connection#close()} when done with this mocked Connection. - */ - public static ConnectionImplementation getMockedConnectionAndDecorate(final Configuration conf, - final AdminProtos.AdminService.BlockingInterface admin, - final ClientProtos.ClientService.BlockingInterface client, final ServerName sn, - final RegionInfo hri) throws IOException { - ConnectionImplementation c = Mockito.mock(ConnectionImplementation.class); - Mockito.when(c.getConfiguration()).thenReturn(conf); - Mockito.doNothing().when(c).close(); - // Make it so we return a particular location when asked. - final HRegionLocation loc = new HRegionLocation(hri, sn); - Mockito.when(c.getRegionLocation((TableName) Mockito.any(), - (byte[]) Mockito.any(), Mockito.anyBoolean())). - thenReturn(loc); - Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())). - thenReturn(loc); - Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any(), - Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt())) - .thenReturn(new RegionLocations(loc)); - if (admin != null) { - // If a call to getAdmin, return this implementation. - Mockito.when(c.getAdmin(Mockito.any())). - thenReturn(admin); - } - if (client != null) { - // If a call to getClient, return this client. - Mockito.when(c.getClient(Mockito.any())). - thenReturn(client); - } - NonceGenerator ng = Mockito.mock(NonceGenerator.class); - Mockito.when(c.getNonceGenerator()).thenReturn(ng); - Mockito.when(c.getAsyncProcess()).thenReturn( - new AsyncProcess(c, conf, RpcRetryingCallerFactory.instantiate(conf), - RpcControllerFactory.instantiate(conf))); - Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn( - RpcRetryingCallerFactory.instantiate(conf, - RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null)); - Mockito.when(c.getRpcControllerFactory()).thenReturn(Mockito.mock(RpcControllerFactory.class)); - Table t = Mockito.mock(Table.class); - Mockito.when(c.getTable((TableName)Mockito.any())).thenReturn(t); - ResultScanner rs = Mockito.mock(ResultScanner.class); - Mockito.when(t.getScanner((Scan)Mockito.any())).thenReturn(rs); - return c; - } - - /** - * Get a Mockito spied-upon {@link ConnectionImplementation} that goes with the passed - * conf configuration instance. Be sure to shutdown the connection when done by - * calling {@link Connection#close()} else it will stick around; this is probably not what you - * want. - * @param conf configuration - * @return ClusterConnection object for conf - */ - public static ConnectionImplementation getSpiedConnection(final Configuration conf) - throws IOException { - ConnectionImplementation connection = - Mockito.spy(new ConnectionImplementation(conf, null, null)); - return connection; - } - - /** - * This coproceesor sleep 2s at first increment/append rpc call. - */ - public static class SleepAtFirstRpcCall implements RegionCoprocessor, RegionObserver { - static final AtomicLong ct = new AtomicLong(0); - static final String SLEEP_TIME_CONF_KEY = - "hbase.coprocessor.SleepAtFirstRpcCall.sleepTime"; - static final long DEFAULT_SLEEP_TIME = 2000; - static final AtomicLong sleepTime = new AtomicLong(DEFAULT_SLEEP_TIME); - - @Override - public Optional getRegionObserver() { - return Optional.of(this); - } - - public SleepAtFirstRpcCall() { - } - - @Override - public void postOpen(ObserverContext c) { - RegionCoprocessorEnvironment env = c.getEnvironment(); - Configuration conf = env.getConfiguration(); - sleepTime.set(conf.getLong(SLEEP_TIME_CONF_KEY, DEFAULT_SLEEP_TIME)); - } - - @Override - public Result postIncrement(final ObserverContext e, - final Increment increment, final Result result) throws IOException { - if (ct.incrementAndGet() == 1) { - Threads.sleep(sleepTime.get()); - } - return result; - } - - @Override - public Result postAppend(final ObserverContext e, - final Append append, final Result result) throws IOException { - if (ct.incrementAndGet() == 1) { - Threads.sleep(sleepTime.get()); - } - return result; - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SleepAtFirstRpcCall.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SleepAtFirstRpcCall.java new file mode 100644 index 000000000000..2ba54bfd2a3a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SleepAtFirstRpcCall.java @@ -0,0 +1,71 @@ +/** + * 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.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.util.Threads; + +/** + * This coproceesor sleep 2s at first increment/append rpc call. + */ +public class SleepAtFirstRpcCall implements RegionCoprocessor, RegionObserver { + static final AtomicLong ct = new AtomicLong(0); + static final String SLEEP_TIME_CONF_KEY = "hbase.coprocessor.SleepAtFirstRpcCall.sleepTime"; + static final long DEFAULT_SLEEP_TIME = 2000; + static final AtomicLong sleepTime = new AtomicLong(DEFAULT_SLEEP_TIME); + + @Override + public Optional getRegionObserver() { + return Optional.of(this); + } + + public SleepAtFirstRpcCall() { + } + + @Override + public void postOpen(ObserverContext c) { + RegionCoprocessorEnvironment env = c.getEnvironment(); + Configuration conf = env.getConfiguration(); + sleepTime.set(conf.getLong(SLEEP_TIME_CONF_KEY, DEFAULT_SLEEP_TIME)); + } + + @Override + public Result postIncrement(final ObserverContext e, + final Increment increment, final Result result) throws IOException { + if (ct.incrementAndGet() == 1) { + Threads.sleep(sleepTime.get()); + } + return result; + } + + @Override + public Result postAppend(final ObserverContext e, + final Append append, final Result result) throws IOException { + if (ct.incrementAndGet() == 1) { + Threads.sleep(sleepTime.get()); + } + return result; + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index 538917d15386..7cc8e9e5e434 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -73,8 +72,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.io.Closeables; - import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; @@ -91,7 +88,6 @@ public class TestAdmin1 { private static final Logger LOG = LoggerFactory.getLogger(TestAdmin1.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static ConnectionImplementation CONN; private static Admin ADMIN; @Rule @@ -105,13 +101,10 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true); TEST_UTIL.startMiniCluster(3); ADMIN = TEST_UTIL.getAdmin(); - CONN = ConnectionFactory.createConnectionImpl(TEST_UTIL.getConfiguration(), null, - UserProvider.instantiate(TEST_UTIL.getConfiguration()).getCurrent()); } @AfterClass public static void tearDownAfterClass() throws Exception { - Closeables.close(CONN, true); TEST_UTIL.shutdownMiniCluster(); } @@ -642,9 +635,9 @@ public void testOnlineChangeTableSchema() throws IOException, InterruptedExcepti assertFalse(ADMIN.tableExists(tableName)); } - private void verifyRoundRobinDistribution(ConnectionImplementation c, - RegionLocator regionLocator, int expectedRegions) throws IOException { - int numRS = c.getCurrentNrHRS(); + private void verifyRoundRobinDistribution(RegionLocator regionLocator, int expectedRegions) + throws IOException { + int numRS = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().size(); List regions = regionLocator.getAllRegionLocations(); Map> server2Regions = new HashMap<>(); for (HRegionLocation loc : regions) { @@ -788,7 +781,7 @@ public void testCreateTableWithRegions() throws IOException, InterruptedExceptio assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8])); assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0); - verifyRoundRobinDistribution(CONN, l, expectedRegions); + verifyRoundRobinDistribution(l, expectedRegions); } @@ -848,7 +841,7 @@ public void testCreateTableWithRegions() throws IOException, InterruptedExceptio assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 })); assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0); - verifyRoundRobinDistribution(CONN, l, expectedRegions); + verifyRoundRobinDistribution(l, expectedRegions); } // Try once more with something that divides into something infinite @@ -871,7 +864,7 @@ public void testCreateTableWithRegions() throws IOException, InterruptedExceptio "but only found " + regions.size(), expectedRegions, regions.size()); System.err.println("Found " + regions.size() + " regions"); - verifyRoundRobinDistribution(CONN, l, expectedRegions); + verifyRoundRobinDistribution(l, expectedRegions); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 2644061595e6..867f10b2456f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -30,7 +30,6 @@ import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -46,9 +45,7 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.Waiter.Predicate; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.constraint.ConstraintException; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.regionserver.HRegion; @@ -74,6 +71,7 @@ import org.apache.hbase.thirdparty.com.google.common.io.Closeables; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; /** * Class to test HBaseAdmin. @@ -89,7 +87,7 @@ public class TestAdmin2 { private static final Logger LOG = LoggerFactory.getLogger(TestAdmin2.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static ConnectionImplementation CONN; + private static Connection CONN; private static Admin ADMIN; @Rule @@ -104,7 +102,7 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 30); TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true); TEST_UTIL.startMiniCluster(3); - CONN = ConnectionFactory.createConnectionImpl(TEST_UTIL.getConfiguration(), null, + CONN = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), null, UserProvider.instantiate(TEST_UTIL.getConfiguration()).getCurrent()); ADMIN = TEST_UTIL.getAdmin(); } @@ -549,30 +547,6 @@ private HRegionServer startAndWriteData(TableName tableName, byte[] value) return regionServer; } - /** - * Check that we have an exception if the cluster is not there. - */ - @Test - public void testCheckHBaseAvailableWithoutCluster() { - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - - // Change the ZK address to go to something not used. - conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, - conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 9999)+10); - - long start = System.currentTimeMillis(); - try { - HBaseAdmin.available(conf); - assertTrue(false); - } catch (ZooKeeperConnectionException ignored) { - } catch (IOException ignored) { - } - long end = System.currentTimeMillis(); - - LOG.info("It took "+(end-start)+" ms to find out that" + - " HBase was not available"); - } - @Test public void testDisableCatalogTable() throws Exception { try { @@ -752,10 +726,12 @@ public void testGetRegionInfo() throws Exception { long expectedStoreFilesSize = store.getStorefilesSize(); Assert.assertNotNull(store); Assert.assertEquals(expectedStoreFilesSize, store.getSize()); - HBaseRpcController controller = CONN.getRpcControllerFactory().newController(); for (int i = 0; i < 10; i++) { - RegionInfo ri = - ProtobufUtil.getRegionInfo(controller, CONN.getAdmin(rs.getServerName()), regionName); + RegionInfo ri = ProtobufUtil + .toRegionInfo(TEST_UTIL.getAsyncConnection().getRegionServerAdmin(rs.getServerName()) + .getRegionInfo(RequestConverter.buildGetRegionInfoRequest(regionName)).get() + .getRegionInfo()); + Assert.assertEquals(region.getRegionInfo(), ri); // Make sure that the store size is still the actual file system's store size. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java index 73529c084ac9..a02ee90fd023 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBase.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import java.io.IOException; import java.util.Arrays; @@ -26,7 +26,6 @@ import java.util.concurrent.ForkJoinPool; import java.util.function.Supplier; import java.util.regex.Pattern; - import org.apache.commons.io.IOUtils; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java index b1e527aba062..f5df30dbba2a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminBuilder.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.fail; import java.io.IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java index 6c34c1f74fe8..94eedca826fe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java index 22a09ebf554b..ab6515321c30 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi2.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java index 8de32b86a347..297169ff9ca7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNamespaceAdminApi.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java index 83845186021d..103f022c7150 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java index 07d436cbe52b..fbf38a0effa9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncQuotaAdminApi.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java index b68ca01e7b63..506f8e75f6fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.startsWith; import static org.junit.Assert.assertEquals; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java index b22caa2a5b7a..398b97ac2dea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApiWithClusters.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER_COUNT_KEY; +import static org.apache.hadoop.hbase.client.AsyncConnectionConfiguration.START_LOG_ERRORS_AFTER_COUNT_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java index e046afa179b9..6fdd4624ca79 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCIBadHostname.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.AfterClass; @@ -34,25 +33,24 @@ import org.junit.experimental.categories.Category; /** - * Tests that we fail fast when hostname resolution is not working and do not cache - * unresolved InetSocketAddresses. + * Tests that we fail fast when hostname resolution is not working and do not cache unresolved + * InetSocketAddresses. */ -@Category({MediumTests.class, ClientTests.class}) +@Category({ MediumTests.class, ClientTests.class }) public class TestCIBadHostname { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestCIBadHostname.class); + HBaseClassTestRule.forClass(TestCIBadHostname.class); - private static HBaseTestingUtility TEST_UTIL; - private static ConnectionImplementation CONN; + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static AsyncConnectionImpl CONN; @BeforeClass public static void setupBeforeClass() throws Exception { - TEST_UTIL = HBaseTestingUtility.createLocalHTU(); TEST_UTIL.startMiniCluster(); - CONN = ConnectionFactory.createConnectionImpl(TEST_UTIL.getConfiguration(), null, - UserProvider.instantiate(TEST_UTIL.getConfiguration()).getCurrent()); + CONN = (AsyncConnectionImpl) ConnectionFactory + .createAsyncConnection(TEST_UTIL.getConfiguration()).get(); } @AfterClass @@ -66,17 +64,16 @@ public void testGetAdminBadHostname() throws Exception { // verify that we can get an instance with the cluster hostname ServerName master = TEST_UTIL.getHBaseCluster().getMaster().getServerName(); try { - CONN.getAdmin(master); + CONN.getAdminStub(master); } catch (UnknownHostException uhe) { fail("Obtaining admin to the cluster master should have succeeded"); } // test that we fail to get a client to an unresolvable hostname, which // means it won't be cached - ServerName badHost = - ServerName.valueOf("unknownhost.invalid:" + HConstants.DEFAULT_MASTER_PORT, - System.currentTimeMillis()); - CONN.getAdmin(badHost); + ServerName badHost = ServerName.valueOf("unknownhost.invalid:" + HConstants.DEFAULT_MASTER_PORT, + System.currentTimeMillis()); + CONN.getAdminStub(badHost); fail("Obtaining admin to unresolvable hostname should have failed"); } @@ -85,17 +82,16 @@ public void testGetClientBadHostname() throws Exception { // verify that we can get an instance with the cluster hostname ServerName rs = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName(); try { - CONN.getClient(rs); + CONN.getRegionServerStub(rs); } catch (UnknownHostException uhe) { fail("Obtaining client to the cluster regionserver should have succeeded"); } // test that we fail to get a client to an unresolvable hostname, which // means it won't be cached - ServerName badHost = - ServerName.valueOf("unknownhost.invalid:" + HConstants.DEFAULT_REGIONSERVER_PORT, - System.currentTimeMillis()); - CONN.getAdmin(badHost); + ServerName badHost = ServerName.valueOf( + "unknownhost.invalid:" + HConstants.DEFAULT_REGIONSERVER_PORT, System.currentTimeMillis()); + CONN.getRegionServerStub(badHost); fail("Obtaining client to unresolvable hostname should have failed"); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java index d91491286a54..1c8480931ef3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java @@ -17,16 +17,12 @@ */ package org.apache.hadoop.hbase.client; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.junit.Before; @@ -86,60 +82,4 @@ public void testRpcRetryingCallerSleep() throws Exception { } } } - - @Test - public void testCallableSleep() throws Exception { - long pauseTime; - long baseTime = 100; - final TableName tableName = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(tableName, FAM_NAM); - try (ConnectionImplementation conn = - ConnectionFactory.createConnectionImpl(TEST_UTIL.getConfiguration(), null, - UserProvider.instantiate(TEST_UTIL.getConfiguration()).getCurrent())) { - ClientServiceCallable regionServerCallable = new ClientServiceCallable(conn, - tableName, FAM_NAM, new RpcControllerFactory(TEST_UTIL.getConfiguration()).newController(), - HConstants.PRIORITY_UNSET) { - @Override - protected Object rpcCall() throws Exception { - return null; - } - }; - - regionServerCallable.prepare(false); - for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) { - pauseTime = regionServerCallable.sleep(baseTime, i); - assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i])); - assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f)); - } - RegionAdminServiceCallable regionAdminServiceCallable = - new RegionAdminServiceCallable(conn, - new RpcControllerFactory(TEST_UTIL.getConfiguration()), tableName, FAM_NAM) { - @Override - public Object call(HBaseRpcController controller) throws Exception { - return null; - } - }; - - regionAdminServiceCallable.prepare(false); - for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) { - pauseTime = regionAdminServiceCallable.sleep(baseTime, i); - assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i])); - assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f)); - } - - try (MasterCallable masterCallable = - new MasterCallable(conn, new RpcControllerFactory(TEST_UTIL.getConfiguration())) { - @Override - protected Object rpcCall() throws Exception { - return null; - } - }) { - for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) { - pauseTime = masterCallable.sleep(baseTime, i); - assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i])); - assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f)); - } - } - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java deleted file mode 100644 index b8994a84dc3b..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java +++ /dev/null @@ -1,109 +0,0 @@ -/** - * 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.util.ArrayList; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; -import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.io.Closeables; - -@Category({ MediumTests.class, ClientTests.class }) -public class TestClientPushback extends ClientPushbackTestBase { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestClientPushback.class); - - private ConnectionImplementation conn; - - private BufferedMutatorImpl mutator; - - @Before - public void setUp() throws IOException { - conn = - (ConnectionImplementation) ConnectionFactory.createConnectionImpl(UTIL.getConfiguration(), - null, UserProvider.instantiate(UTIL.getConfiguration()).getCurrent()); - mutator = (BufferedMutatorImpl) conn.getBufferedMutator(tableName); - } - - @After - public void tearDown() throws IOException { - Closeables.close(mutator, true); - Closeables.close(conn, true); - } - - @Override - protected ClientBackoffPolicy getBackoffPolicy() throws IOException { - return conn.getBackoffPolicy(); - } - - @Override - protected ServerStatisticTracker getStatisticsTracker() throws IOException { - return conn.getStatisticsTracker(); - } - - @Override - protected MetricsConnection getConnectionMetrics() throws IOException { - return conn.getConnectionMetrics(); - } - - @Override - protected void mutate(Put put) throws IOException { - mutator.mutate(put); - mutator.flush(); - } - - @Override - protected void mutate(Put put, AtomicLong endTime, CountDownLatch latch) throws IOException { - // Reach into the connection and submit work directly to AsyncProcess so we can - // monitor how long the submission was delayed via a callback - List ops = new ArrayList<>(1); - ops.add(put); - Batch.Callback callback = (byte[] r, byte[] row, Result result) -> { - endTime.set(EnvironmentEdgeManager.currentTime()); - latch.countDown(); - }; - 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); - } - - @Override - protected void mutateRow(RowMutations mutations) throws IOException { - try (Table table = conn.getTable(tableName)) { - table.mutateRow(mutations); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index 28a53cd7433b..bd1f7ccd787b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -24,7 +24,7 @@ import java.net.SocketAddress; import java.net.SocketTimeoutException; import java.net.UnknownHostException; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ipc.AbstractRpcClient; import org.apache.hadoop.hbase.ipc.BlockingRpcClient; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcClientFactory; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -48,50 +49,44 @@ import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel; import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; -@Category({MediumTests.class, ClientTests.class}) +@Category({ MediumTests.class, ClientTests.class }) public class TestClientTimeouts { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestClientTimeouts.class); + HBaseClassTestRule.forClass(TestClientTimeouts.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); protected static int SLAVES = 1; - /** - * @throws java.lang.Exception - */ @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(SLAVES); // Set the custom RPC client with random timeouts as the client - TEST_UTIL.getConfiguration().set( - RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, - RandomTimeoutRpcClient.class.getName()); + TEST_UTIL.getConfiguration().set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, + RandomTimeoutRpcClient.class.getName()); } - /** - * @throws java.lang.Exception - */ @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } /** - * Test that a client that fails an RPC to the master retries properly and - * doesn't throw any unexpected exceptions. - * @throws Exception + * Test that a client that fails an RPC to the master retries properly and doesn't throw any + * unexpected exceptions. */ @Test public void testAdminTimeout() throws Exception { boolean lastFailed = false; - int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get(); + int initialInvocations = invokations.get(); RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); try { for (int i = 0; i < 5 || (lastFailed && i < 100); ++i) { @@ -104,29 +99,27 @@ public void testAdminTimeout() throws Exception { try { connection = ConnectionFactory.createConnection(conf); admin = connection.getAdmin(); - // run some admin commands - HBaseAdmin.available(conf); admin.balancerSwitch(false, false); } catch (MasterNotRunningException ex) { // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get - // a MasterNotRunningException. It's a bug if we get other exceptions. + // a MasterNotRunningException. It's a bug if we get other exceptions. lastFailed = true; } finally { - if(admin != null) { + if (admin != null) { admin.close(); if (admin.getConnection().isClosed()) { rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); } } - if(connection != null) { + if (connection != null) { connection.close(); } } } // Ensure the RandomTimeoutRpcEngine is actually being used. assertFalse(lastFailed); - assertTrue(RandomTimeoutBlockingRpcChannel.invokations.get() > initialInvocations); + assertTrue(invokations.get() > initialInvocations); } finally { rpcClient.close(); } @@ -143,38 +136,67 @@ public RandomTimeoutRpcClient(Configuration conf, String clusterId, SocketAddres // Return my own instance, one that does random timeouts @Override - public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, - User ticket, int rpcTimeout) throws UnknownHostException { + public BlockingRpcChannel createBlockingRpcChannel(ServerName sn, User ticket, int rpcTimeout) + throws UnknownHostException { return new RandomTimeoutBlockingRpcChannel(this, sn, ticket, rpcTimeout); } + + @Override + public RpcChannel createRpcChannel(ServerName sn, User ticket, int rpcTimeout) + throws UnknownHostException { + return new RandomTimeoutRpcChannel(this, sn, ticket, rpcTimeout); + } } + private static AtomicInteger invokations = new AtomicInteger(); + + private static final double CHANCE_OF_TIMEOUT = 0.3; + /** * Blocking rpc channel that goes via hbase rpc. */ - static class RandomTimeoutBlockingRpcChannel + private static class RandomTimeoutBlockingRpcChannel extends AbstractRpcClient.BlockingRpcChannelImplementation { - private static final Random RANDOM = new Random(System.currentTimeMillis()); - public static final double CHANCE_OF_TIMEOUT = 0.3; - private static AtomicInteger invokations = new AtomicInteger(); - RandomTimeoutBlockingRpcChannel(final BlockingRpcClient rpcClient, final ServerName sn, - final User ticket, final int rpcTimeout) { + RandomTimeoutBlockingRpcChannel(BlockingRpcClient rpcClient, ServerName sn, User ticket, + int rpcTimeout) { super(rpcClient, new InetSocketAddress(sn.getHostname(), sn.getPort()), ticket, rpcTimeout); } @Override - public Message callBlockingMethod(MethodDescriptor md, - RpcController controller, Message param, Message returnType) - throws ServiceException { + public Message callBlockingMethod(MethodDescriptor md, RpcController controller, Message param, + Message returnType) throws ServiceException { invokations.getAndIncrement(); - if (RANDOM.nextFloat() < CHANCE_OF_TIMEOUT) { + if (ThreadLocalRandom.current().nextFloat() < CHANCE_OF_TIMEOUT) { // throw a ServiceException, becuase that is the only exception type that - // {@link ProtobufRpcEngine} throws. If this RpcEngine is used with a different + // {@link ProtobufRpcEngine} throws. If this RpcEngine is used with a different // "actual" type, this may not properly mimic the underlying RpcEngine. throw new ServiceException(new SocketTimeoutException("fake timeout")); } return super.callBlockingMethod(md, controller, param, returnType); } } + + private static class RandomTimeoutRpcChannel extends AbstractRpcClient.RpcChannelImplementation { + + RandomTimeoutRpcChannel(AbstractRpcClient rpcClient, ServerName sn, User ticket, + int rpcTimeout) throws UnknownHostException { + super(rpcClient, new InetSocketAddress(sn.getHostname(), sn.getPort()), ticket, rpcTimeout); + } + + @Override + public void callMethod(MethodDescriptor md, RpcController controller, Message param, + Message returnType, RpcCallback done) { + invokations.getAndIncrement(); + if (ThreadLocalRandom.current().nextFloat() < CHANCE_OF_TIMEOUT) { + // throw a ServiceException, because that is the only exception type that + // {@link ProtobufRpcEngine} throws. If this RpcEngine is used with a different + // "actual" type, this may not properly mimic the underlying RpcEngine. + ((HBaseRpcController) controller).setFailed(new SocketTimeoutException("fake timeout")); + done.run(null); + return; + } + super.callMethod(md, controller, param, returnType, done); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java new file mode 100644 index 000000000000..8dd4709c1c91 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnection.java @@ -0,0 +1,356 @@ +/** + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is for testing {@link Connection}. + */ +@Category({ LargeTests.class }) +public class TestConnection { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestConnection.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestConnection.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static final byte[] FAM_NAM = Bytes.toBytes("f"); + private static final byte[] ROW = Bytes.toBytes("bbb"); + private static final int RPC_RETRY = 5; + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true); + // Up the handlers; this test needs more than usual. + TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10); + TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY); + TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 3); + TEST_UTIL.startMiniCluster(2); + + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * Naive test to check that Connection#getAdmin returns a properly constructed HBaseAdmin object + * @throws IOException Unable to construct admin + */ + @Test + public void testAdminFactory() throws IOException { + Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + Admin admin = con1.getAdmin(); + assertTrue(admin.getConnection() == con1); + assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration()); + con1.close(); + } + + /** + * Test that we can handle connection close: it will trigger a retry, but the calls will finish. + */ + @Test + public void testConnectionCloseAllowsInterrupt() throws Exception { + testConnectionClose(true); + } + + @Test + public void testConnectionNotAllowsInterrupt() throws Exception { + testConnectionClose(false); + } + + private void testConnectionClose(boolean allowsInterrupt) throws Exception { + TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt); + TEST_UTIL.createTable(tableName, FAM_NAM).close(); + + boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true); + + Configuration c2 = new Configuration(TEST_UTIL.getConfiguration()); + // We want to work on a separate connection. + c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); + c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 100); // retry a lot + c2.setInt(HConstants.HBASE_CLIENT_PAUSE, 1); // don't wait between retries. + c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire + c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt); + // to avoid the client to be stuck when do the Get + c2.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, 10000); + c2.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 10000); + c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 5000); + + Connection connection = ConnectionFactory.createConnection(c2); + final Table table = connection.getTable(tableName); + + Put put = new Put(ROW); + put.addColumn(FAM_NAM, ROW, ROW); + table.put(put); + + // 4 steps: ready=0; doGets=1; mustStop=2; stopped=3 + final AtomicInteger step = new AtomicInteger(0); + + final AtomicReference failed = new AtomicReference<>(null); + Thread t = new Thread("testConnectionCloseThread") { + @Override + public void run() { + int done = 0; + try { + step.set(1); + while (step.get() == 1) { + Get get = new Get(ROW); + table.get(get); + done++; + if (done % 100 == 0) { + LOG.info("done=" + done); + } + // without the sleep, will cause the exception for too many files in + // org.apache.hadoop.hdfs.server.datanode.DataXceiver + Thread.sleep(100); + } + } catch (Throwable t) { + failed.set(t); + LOG.error(t.toString(), t); + } + step.set(3); + } + }; + t.start(); + TEST_UTIL.waitFor(20000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return step.get() == 1; + } + }); + + ServerName sn; + try (RegionLocator rl = connection.getRegionLocator(tableName)) { + sn = rl.getRegionLocation(ROW).getServerName(); + } + + RpcClient rpcClient = ((AsyncConnectionImpl) connection.toAsyncConnection()).rpcClient; + + LOG.info("Going to cancel connections. connection=" + connection.toString() + ", sn=" + sn); + for (int i = 0; i < 5000; i++) { + rpcClient.cancelConnections(sn); + Thread.sleep(5); + } + + step.compareAndSet(1, 2); + // The test may fail here if the thread doing the gets is stuck. The way to find + // out what's happening is to look for the thread named 'testConnectionCloseThread' + TEST_UTIL.waitFor(40000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return step.get() == 3; + } + }); + table.close(); + connection.close(); + Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null); + TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true); + } + + /** + * Test that connection can become idle without breaking everything. + */ + @Test + public void testConnectionIdle() throws Exception { + final TableName tableName = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(tableName, FAM_NAM).close(); + int idleTime = 20000; + boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true); + + Configuration c2 = new Configuration(TEST_UTIL.getConfiguration()); + // We want to work on a separate connection. + c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); + c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed + c2.setInt(RpcClient.IDLE_TIME, idleTime); + + Connection connection = ConnectionFactory.createConnection(c2); + final Table table = connection.getTable(tableName); + + Put put = new Put(ROW); + put.addColumn(FAM_NAM, ROW, ROW); + table.put(put); + + ManualEnvironmentEdge mee = new ManualEnvironmentEdge(); + mee.setValue(System.currentTimeMillis()); + EnvironmentEdgeManager.injectEdge(mee); + LOG.info("first get"); + table.get(new Get(ROW)); + + LOG.info("first get - changing the time & sleeping"); + mee.incValue(idleTime + 1000); + Thread.sleep(1500); // we need to wait a little for the connection to be seen as idle. + // 1500 = sleep time in RpcClient#waitForWork + a margin + + LOG.info("second get - connection has been marked idle in the middle"); + // To check that the connection actually became idle would need to read some private + // fields of RpcClient. + table.get(new Get(ROW)); + mee.incValue(idleTime + 1000); + + LOG.info("third get - connection is idle, but the reader doesn't know yet"); + // We're testing here a special case: + // time limit reached BUT connection not yet reclaimed AND a new call. + // in this situation, we don't close the connection, instead we use it immediately. + // If we're very unlucky we can have a race condition in the test: the connection is already + // under closing when we do the get, so we have an exception, and we don't retry as the + // retry number is 1. The probability is very very low, and seems acceptable for now. It's + // a test issue only. + table.get(new Get(ROW)); + + LOG.info("we're done - time will change back"); + + table.close(); + + connection.close(); + EnvironmentEdgeManager.reset(); + TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true); + } + + @Test + public void testClosing() throws Exception { + Configuration configuration = new Configuration(TEST_UTIL.getConfiguration()); + configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, + String.valueOf(ThreadLocalRandom.current().nextInt())); + + // as connection caching is going away, now we're just testing + // that closed connection does actually get closed. + + Connection c1 = ConnectionFactory.createConnection(configuration); + Connection c2 = ConnectionFactory.createConnection(configuration); + // no caching, different connections + assertTrue(c1 != c2); + + // closing independently + c1.close(); + assertTrue(c1.isClosed()); + assertFalse(c2.isClosed()); + + c2.close(); + assertTrue(c2.isClosed()); + } + + /** + * Trivial test to verify that nobody messes with + * {@link ConnectionFactory#createConnection(Configuration)} + */ + @Test + public void testCreateConnection() throws Exception { + Configuration configuration = TEST_UTIL.getConfiguration(); + Connection c1 = ConnectionFactory.createConnection(configuration); + Connection c2 = ConnectionFactory.createConnection(configuration); + // created from the same configuration, yet they are different + assertTrue(c1 != c2); + assertTrue(c1.getConfiguration() == c2.getConfiguration()); + } + + @Test + public void testConnectionRideOverClusterRestart() throws IOException, InterruptedException { + Configuration config = new Configuration(TEST_UTIL.getConfiguration()); + + final TableName tableName = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(tableName, new byte[][] { FAM_NAM }).close(); + + Connection connection = ConnectionFactory.createConnection(config); + Table table = connection.getTable(tableName); + + // this will cache the meta location and table's region location + table.get(new Get(Bytes.toBytes("foo"))); + + // restart HBase + TEST_UTIL.shutdownMiniHBaseCluster(); + TEST_UTIL.restartHBaseCluster(2); + // this should be able to discover new locations for meta and table's region + table.get(new Get(Bytes.toBytes("foo"))); + TEST_UTIL.deleteTable(tableName); + table.close(); + connection.close(); + } + + @Test + public void testLocateRegionsWithRegionReplicas() throws IOException { + int regionReplication = 3; + byte[] family = Bytes.toBytes("cf"); + TableName tableName = TableName.valueOf(name.getMethodName()); + + // Create a table with region replicas + TableDescriptorBuilder builder = + TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)); + TEST_UTIL.getAdmin().createTable(builder.build()); + + try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); + RegionLocator locator = conn.getRegionLocator(tableName)) { + // Get locations of the regions of the table + List locations = locator.getAllRegionLocations(); + + // The size of the returned locations should be 3 + assertEquals(regionReplication, locations.size()); + + // The replicaIds of the returned locations should be 0, 1 and 2 + Set expectedReplicaIds = + IntStream.range(0, regionReplication).boxed().collect(Collectors.toSet()); + for (HRegionLocation location : locations) { + assertTrue(expectedReplicaIds.remove(location.getRegion().getReplicaId())); + } + } finally { + TEST_UTIL.deleteTable(tableName); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java deleted file mode 100644 index 148d99f60900..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java +++ /dev/null @@ -1,1096 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.lang.reflect.Field; -import java.lang.reflect.Modifier; -import java.net.SocketTimeoutException; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -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.Waiter; -import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.exceptions.RegionMovedException; -import org.apache.hadoop.hbase.filter.Filter; -import org.apache.hadoop.hbase.filter.FilterBase; -import org.apache.hadoop.hbase.ipc.RpcClient; -import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.JVMClusterUtil; -import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; -import org.apache.hadoop.hbase.util.Threads; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; - -/** - * This class is for testing HBaseConnectionManager features - *

- * Will be removed in the future, ignore. - */ -@Ignore -@Category({LargeTests.class}) -public class TestConnectionImplementation { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestConnectionImplementation.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestConnectionImplementation.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static final TableName TABLE_NAME = - TableName.valueOf("test"); - private static final TableName TABLE_NAME1 = - TableName.valueOf("test1"); - private static final TableName TABLE_NAME2 = - TableName.valueOf("test2"); - private static final TableName TABLE_NAME3 = - TableName.valueOf("test3"); - private static final byte[] FAM_NAM = Bytes.toBytes("f"); - private static final byte[] ROW = Bytes.toBytes("bbb"); - private static final byte[] ROW_X = Bytes.toBytes("xxx"); - private static final int RPC_RETRY = 5; - - @Rule - public TestName name = new TestName(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true); - // Up the handlers; this test needs more than usual. - TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10); - TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY); - TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 3); - TEST_UTIL.startMiniCluster(2); - - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testClusterConnection() throws IOException { - ThreadPoolExecutor otherPool = new ThreadPoolExecutor(1, 1, - 5, TimeUnit.SECONDS, - new SynchronousQueue<>(), - Threads.newDaemonThreadFactory("test-hcm")); - - Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Connection con2 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), otherPool); - // make sure the internally created ExecutorService is the one passed - assertTrue(otherPool == ((ConnectionImplementation) con2).getCurrentBatchPool()); - - final TableName tableName = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(tableName, FAM_NAM).close(); - Table table = con1.getTable(tableName, otherPool); - - ExecutorService pool = null; - - if(table instanceof HTable) { - HTable t = (HTable) table; - // make sure passing a pool to the getTable does not trigger creation of an internal pool - assertNull("Internal Thread pool should be null", - ((ConnectionImplementation) con1).getCurrentBatchPool()); - // table should use the pool passed - assertTrue(otherPool == t.getPool()); - t.close(); - - t = (HTable) con2.getTable(tableName); - // table should use the connectin's internal pool - assertTrue(otherPool == t.getPool()); - t.close(); - - t = (HTable) con2.getTable(tableName); - // try other API too - assertTrue(otherPool == t.getPool()); - t.close(); - - t = (HTable) con2.getTable(tableName); - // try other API too - assertTrue(otherPool == t.getPool()); - t.close(); - - t = (HTable) con1.getTable(tableName); - pool = ((ConnectionImplementation) con1).getCurrentBatchPool(); - // make sure an internal pool was created - assertNotNull("An internal Thread pool should have been created", pool); - // and that the table is using it - assertTrue(t.getPool() == pool); - t.close(); - - t = (HTable) con1.getTable(tableName); - // still using the *same* internal pool - assertTrue(t.getPool() == pool); - t.close(); - } else { - table.close(); - } - - con1.close(); - - // if the pool was created on demand it should be closed upon connection close - if(pool != null) { - assertTrue(pool.isShutdown()); - } - - con2.close(); - // if the pool is passed, it is not closed - assertFalse(otherPool.isShutdown()); - otherPool.shutdownNow(); - } - - /** - * Naive test to check that Connection#getAdmin returns a properly constructed HBaseAdmin object - * @throws IOException Unable to construct admin - */ - @Test - public void testAdminFactory() throws IOException { - Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Admin admin = con1.getAdmin(); - assertTrue(admin.getConnection() == con1); - assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration()); - con1.close(); - } - - // Fails too often! Needs work. HBASE-12558 - // May only fail on non-linux machines? E.g. macosx. - @Ignore @Test (expected = RegionServerStoppedException.class) - // Depends on mulitcast messaging facility that seems broken in hbase2 - // See HBASE-19261 "ClusterStatusPublisher where Master could optionally broadcast notice of - // dead servers is broke" - public void testClusterStatus() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - byte[] cf = Bytes.toBytes("cf"); - byte[] rk = Bytes.toBytes("rk1"); - - JVMClusterUtil.RegionServerThread rs = TEST_UTIL.getHBaseCluster().startRegionServer(); - rs.waitForServerOnline(); - final ServerName sn = rs.getRegionServer().getServerName(); - - Table t = TEST_UTIL.createTable(tableName, cf); - TEST_UTIL.waitTableAvailable(tableName); - TEST_UTIL.waitUntilNoRegionsInTransition(); - - final ConnectionImplementation hci = (ConnectionImplementation)TEST_UTIL.getConnection(); - try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) { - while (l.getRegionLocation(rk).getPort() != sn.getPort()) { - TEST_UTIL.getAdmin().move(l.getRegionLocation(rk).getRegionInfo().getEncodedNameAsBytes(), - sn); - TEST_UTIL.waitUntilNoRegionsInTransition(); - hci.clearRegionCache(tableName); - } - Assert.assertNotNull(hci.clusterStatusListener); - TEST_UTIL.assertRegionOnServer(l.getRegionLocation(rk).getRegionInfo(), sn, 20000); - } - - Put p1 = new Put(rk); - p1.addColumn(cf, Bytes.toBytes("qual"), Bytes.toBytes("val")); - t.put(p1); - - rs.getRegionServer().abort("I'm dead"); - - // We want the status to be updated. That's a least 10 second - TEST_UTIL.waitFor(40000, 1000, true, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return TEST_UTIL.getHBaseCluster().getMaster().getServerManager(). - getDeadServers().isDeadServer(sn); - } - }); - - TEST_UTIL.waitFor(40000, 1000, true, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return hci.clusterStatusListener.isDeadServer(sn); - } - }); - - t.close(); - hci.getClient(sn); // will throw an exception: RegionServerStoppedException - } - - /** - * Test that we can handle connection close: it will trigger a retry, but the calls will finish. - */ - @Test - public void testConnectionCloseAllowsInterrupt() throws Exception { - testConnectionClose(true); - } - - @Test - public void testConnectionNotAllowsInterrupt() throws Exception { - testConnectionClose(false); - } - - private void testConnectionClose(boolean allowsInterrupt) throws Exception { - TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt); - TEST_UTIL.createTable(tableName, FAM_NAM).close(); - - boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true); - - Configuration c2 = new Configuration(TEST_UTIL.getConfiguration()); - // We want to work on a separate connection. - c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 100); // retry a lot - c2.setInt(HConstants.HBASE_CLIENT_PAUSE, 1); // don't wait between retries. - c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire - c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt); - // to avoid the client to be stuck when do the Get - c2.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, 10000); - c2.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 10000); - c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 5000); - - Connection connection = ConnectionFactory.createConnection(c2); - final Table table = connection.getTable(tableName); - - Put put = new Put(ROW); - put.addColumn(FAM_NAM, ROW, ROW); - table.put(put); - - // 4 steps: ready=0; doGets=1; mustStop=2; stopped=3 - final AtomicInteger step = new AtomicInteger(0); - - final AtomicReference failed = new AtomicReference<>(null); - Thread t = new Thread("testConnectionCloseThread") { - @Override - public void run() { - int done = 0; - try { - step.set(1); - while (step.get() == 1) { - Get get = new Get(ROW); - table.get(get); - done++; - if (done % 100 == 0) - LOG.info("done=" + done); - // without the sleep, will cause the exception for too many files in - // org.apache.hadoop.hdfs.server.datanode.DataXceiver - Thread.sleep(100); - } - } catch (Throwable t) { - failed.set(t); - LOG.error(t.toString(), t); - } - step.set(3); - } - }; - t.start(); - TEST_UTIL.waitFor(20000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return step.get() == 1; - } - }); - - ServerName sn; - try(RegionLocator rl = connection.getRegionLocator(tableName)) { - sn = rl.getRegionLocation(ROW).getServerName(); - } - ConnectionImplementation conn = - (ConnectionImplementation) connection; - RpcClient rpcClient = conn.getRpcClient(); - - LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn); - for (int i = 0; i < 5000; i++) { - rpcClient.cancelConnections(sn); - Thread.sleep(5); - } - - step.compareAndSet(1, 2); - // The test may fail here if the thread doing the gets is stuck. The way to find - // out what's happening is to look for the thread named 'testConnectionCloseThread' - TEST_UTIL.waitFor(40000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return step.get() == 3; - } - }); - table.close(); - connection.close(); - Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null); - TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true); - } - - /** - * Test that connection can become idle without breaking everything. - */ - @Test - public void testConnectionIdle() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(tableName, FAM_NAM).close(); - int idleTime = 20000; - boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true); - - Configuration c2 = new Configuration(TEST_UTIL.getConfiguration()); - // We want to work on a separate connection. - c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed - c2.setInt(RpcClient.IDLE_TIME, idleTime); - - Connection connection = ConnectionFactory.createConnection(c2); - final Table table = connection.getTable(tableName); - - Put put = new Put(ROW); - put.addColumn(FAM_NAM, ROW, ROW); - table.put(put); - - ManualEnvironmentEdge mee = new ManualEnvironmentEdge(); - mee.setValue(System.currentTimeMillis()); - EnvironmentEdgeManager.injectEdge(mee); - LOG.info("first get"); - table.get(new Get(ROW)); - - LOG.info("first get - changing the time & sleeping"); - mee.incValue(idleTime + 1000); - Thread.sleep(1500); // we need to wait a little for the connection to be seen as idle. - // 1500 = sleep time in RpcClient#waitForWork + a margin - - LOG.info("second get - connection has been marked idle in the middle"); - // To check that the connection actually became idle would need to read some private - // fields of RpcClient. - table.get(new Get(ROW)); - mee.incValue(idleTime + 1000); - - LOG.info("third get - connection is idle, but the reader doesn't know yet"); - // We're testing here a special case: - // time limit reached BUT connection not yet reclaimed AND a new call. - // in this situation, we don't close the connection, instead we use it immediately. - // If we're very unlucky we can have a race condition in the test: the connection is already - // under closing when we do the get, so we have an exception, and we don't retry as the - // retry number is 1. The probability is very very low, and seems acceptable for now. It's - // a test issue only. - table.get(new Get(ROW)); - - LOG.info("we're done - time will change back"); - - table.close(); - - connection.close(); - EnvironmentEdgeManager.reset(); - TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true); - } - - /** - * Test that the connection to the dead server is cut immediately when we receive the - * notification. - * @throws Exception - */ - @Test - public void testConnectionCut() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - - TEST_UTIL.createTable(tableName, FAM_NAM).close(); - boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true); - - Configuration c2 = new Configuration(TEST_UTIL.getConfiguration()); - // We want to work on a separate connection. - c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - // try only once w/o any retry - c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0); - c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 1000); - - final Connection connection = ConnectionFactory.createConnection(c2); - final Table table = connection.getTable(tableName); - - Put p = new Put(FAM_NAM); - p.addColumn(FAM_NAM, FAM_NAM, FAM_NAM); - table.put(p); - - final ConnectionImplementation hci = (ConnectionImplementation) connection; - - final HRegionLocation loc; - try(RegionLocator rl = connection.getRegionLocator(tableName)) { - loc = rl.getRegionLocation(FAM_NAM); - } - - Get get = new Get(FAM_NAM); - Assert.assertNotNull(table.get(get)); - - get = new Get(FAM_NAM); - get.setFilter(new BlockingFilter()); - - // This thread will mark the server as dead while we're waiting during a get. - Thread t = new Thread() { - @Override - public void run() { - synchronized (syncBlockingFilter) { - try { - syncBlockingFilter.wait(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - hci.clusterStatusListener.deadServerHandler.newDead(loc.getServerName()); - } - }; - - t.start(); - try { - table.get(get); - Assert.fail(); - } catch (IOException expected) { - LOG.debug("Received: " + expected); - Assert.assertFalse(expected instanceof SocketTimeoutException); - Assert.assertFalse(syncBlockingFilter.get()); - } finally { - syncBlockingFilter.set(true); - t.join(); - TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true); - } - - table.close(); - connection.close(); - } - - protected static final AtomicBoolean syncBlockingFilter = new AtomicBoolean(false); - - public static class BlockingFilter extends FilterBase { - @Override - public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { - int i = 0; - while (i++ < 1000 && !syncBlockingFilter.get()) { - synchronized (syncBlockingFilter) { - syncBlockingFilter.notifyAll(); - } - Threads.sleep(100); - } - syncBlockingFilter.set(true); - return false; - } - @Override - public ReturnCode filterCell(final Cell ignored) throws IOException { - return ReturnCode.INCLUDE; - } - - public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException{ - return new BlockingFilter(); - } - } - - /** - * Test that when we delete a location using the first row of a region - * that we really delete it. - * @throws Exception - */ - @Test - public void testRegionCaching() throws Exception { - TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAM_NAM).close(); - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - // test with no retry, or client cache will get updated after the first failure - conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0); - Connection connection = ConnectionFactory.createConnection(conf); - final Table table = connection.getTable(TABLE_NAME); - - TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); - Put put = new Put(ROW); - put.addColumn(FAM_NAM, ROW, ROW); - table.put(put); - - ConnectionImplementation conn = (ConnectionImplementation) connection; - - assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW)); - - // Here we mess with the cached location making it so the region at TABLE_NAME, ROW is at - // a location where the port is current port number +1 -- i.e. a non-existent location. - HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation(); - final int nextPort = loc.getPort() + 1; - conn.updateCachedLocation(loc.getRegionInfo(), loc.getServerName(), - ServerName.valueOf("127.0.0.1", nextPort, - HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP); - Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW) - .getRegionLocation().getPort(), nextPort); - - conn.clearRegionCache(TABLE_NAME, ROW.clone()); - RegionLocations rl = conn.getCachedLocation(TABLE_NAME, ROW); - assertNull("What is this location?? " + rl, rl); - - // We're now going to move the region and check that it works for the client - // First a new put to add the location in the cache - conn.clearRegionCache(TABLE_NAME); - Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME)); - Put put2 = new Put(ROW); - put2.addColumn(FAM_NAM, ROW, ROW); - table.put(put2); - assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW)); - assertNotNull(conn.getCachedLocation(TableName.valueOf(TABLE_NAME.getName()), ROW.clone())); - - TEST_UTIL.getAdmin().balancerSwitch(false, false); - HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster(); - - // We can wait for all regions to be online, that makes log reading easier when debugging - TEST_UTIL.waitUntilNoRegionsInTransition(); - - // Now moving the region to the second server - HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation(); - byte[] regionName = toMove.getRegionInfo().getRegionName(); - byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes(); - - // Choose the other server. - int curServerId = TEST_UTIL.getHBaseCluster().getServerWith(regionName); - int destServerId = curServerId == 0? 1: 0; - - HRegionServer curServer = TEST_UTIL.getHBaseCluster().getRegionServer(curServerId); - HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(destServerId); - - ServerName destServerName = destServer.getServerName(); - - // Check that we are in the expected state - Assert.assertTrue(curServer != destServer); - Assert.assertFalse(curServer.getServerName().equals(destServer.getServerName())); - Assert.assertFalse( toMove.getPort() == destServerName.getPort()); - Assert.assertNotNull(curServer.getOnlineRegion(regionName)); - Assert.assertNull(destServer.getOnlineRegion(regionName)); - Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster(). - getAssignmentManager().hasRegionsInTransition()); - - // Moving. It's possible that we don't have all the regions online at this point, so - // the test must depend only on the region we're looking at. - LOG.info("Move starting region="+toMove.getRegionInfo().getRegionNameAsString()); - TEST_UTIL.getAdmin().move(toMove.getRegionInfo().getEncodedNameAsBytes(), destServerName); - - while (destServer.getOnlineRegion(regionName) == null || - destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) || - curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) || - master.getAssignmentManager().hasRegionsInTransition()) { - // wait for the move to be finished - Thread.sleep(1); - } - - LOG.info("Move finished for region="+toMove.getRegionInfo().getRegionNameAsString()); - - // Check our new state. - Assert.assertNull(curServer.getOnlineRegion(regionName)); - Assert.assertNotNull(destServer.getOnlineRegion(regionName)); - Assert.assertFalse(destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes)); - Assert.assertFalse(curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes)); - - - // Cache was NOT updated and points to the wrong server - Assert.assertFalse( - conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation() - .getPort() == destServerName.getPort()); - - // This part relies on a number of tries equals to 1. - // We do a put and expect the cache to be updated, even if we don't retry - LOG.info("Put starting"); - Put put3 = new Put(ROW); - put3.addColumn(FAM_NAM, ROW, ROW); - try { - table.put(put3); - Assert.fail("Unreachable point"); - } catch (RetriesExhaustedWithDetailsException e) { - LOG.info("Put done, exception caught: " + e.getClass()); - Assert.assertEquals(1, e.getNumExceptions()); - Assert.assertEquals(1, e.getCauses().size()); - Assert.assertArrayEquals(ROW, e.getRow(0).getRow()); - - // Check that we unserialized the exception as expected - Throwable cause = ClientExceptionsUtil.findException(e.getCause(0)); - Assert.assertNotNull(cause); - Assert.assertTrue(cause instanceof RegionMovedException); - } catch (RetriesExhaustedException ree) { - // hbase2 throws RetriesExhaustedException instead of RetriesExhaustedWithDetailsException - // as hbase1 used to do. Keep an eye on this to see if this changed behavior is an issue. - LOG.info("Put done, exception caught: " + ree.getClass()); - Throwable cause = ClientExceptionsUtil.findException(ree.getCause()); - Assert.assertNotNull(cause); - Assert.assertTrue(cause instanceof RegionMovedException); - } - Assert.assertNotNull("Cached connection is null", conn.getCachedLocation(TABLE_NAME, ROW)); - Assert.assertEquals( - "Previous server was " + curServer.getServerName().getHostAndPort(), - destServerName.getPort(), - conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort()); - - Assert.assertFalse(destServer.getRegionsInTransitionInRS() - .containsKey(encodedRegionNameBytes)); - Assert.assertFalse(curServer.getRegionsInTransitionInRS() - .containsKey(encodedRegionNameBytes)); - - // We move it back to do another test with a scan - LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString()); - TEST_UTIL.getAdmin().move(toMove.getRegionInfo().getEncodedNameAsBytes(), - curServer.getServerName()); - - while (curServer.getOnlineRegion(regionName) == null || - destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) || - curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) || - master.getAssignmentManager().hasRegionsInTransition()) { - // wait for the move to be finished - Thread.sleep(1); - } - - // Check our new state. - Assert.assertNotNull(curServer.getOnlineRegion(regionName)); - Assert.assertNull(destServer.getOnlineRegion(regionName)); - LOG.info("Move finished for region=" + toMove.getRegionInfo().getRegionNameAsString()); - - // Cache was NOT updated and points to the wrong server - Assert.assertFalse(conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort() == - curServer.getServerName().getPort()); - - Scan sc = new Scan(); - sc.setStopRow(ROW); - sc.setStartRow(ROW); - - // The scanner takes the max retries from the connection configuration, not the table as - // the put. - TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - - try { - ResultScanner rs = table.getScanner(sc); - while (rs.next() != null) { - } - Assert.fail("Unreachable point"); - } catch (RetriesExhaustedException e) { - LOG.info("Scan done, expected exception caught: " + e.getClass()); - } - - // Cache is updated with the right value. - Assert.assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW)); - Assert.assertEquals( - "Previous server was "+destServer.getServerName().getHostAndPort(), - curServer.getServerName().getPort(), - conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort()); - - TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY); - table.close(); - connection.close(); - } - - /** - * Test that Connection or Pool are not closed when managed externally - * @throws Exception - */ - @Test - public void testConnectionManagement() throws Exception{ - Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM); - Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Table table = conn.getTable(TABLE_NAME1); - table.close(); - assertFalse(conn.isClosed()); - if(table instanceof HTable) { - assertFalse(((HTable) table).getPool().isShutdown()); - } - table = conn.getTable(TABLE_NAME1); - table.close(); - if(table instanceof HTable) { - assertFalse(((HTable) table).getPool().isShutdown()); - } - conn.close(); - if(table instanceof HTable) { - assertTrue(((HTable) table).getPool().isShutdown()); - } - table0.close(); - } - - /** - * Test that stale cache updates don't override newer cached values. - */ - @Test - public void testCacheSeqNums() throws Exception{ - Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME2, FAM_NAM); - Put put = new Put(ROW); - put.addColumn(FAM_NAM, ROW, ROW); - table.put(put); - ConnectionImplementation conn = (ConnectionImplementation) TEST_UTIL.getConnection(); - - HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation(); - assertNotNull(location); - - ServerName anySource = ServerName.valueOf(location.getHostname(), location.getPort() - 1, 0L); - - // Same server as already in cache reporting - overwrites any value despite seqNum. - int nextPort = location.getPort() + 1; - conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(), - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); - location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation(); - Assert.assertEquals(nextPort, location.getPort()); - - // No source specified - same. - nextPort = location.getPort() + 1; - conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(), - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); - location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation(); - Assert.assertEquals(nextPort, location.getPort()); - - // Higher seqNum - overwrites lower seqNum. - nextPort = location.getPort() + 1; - conn.updateCachedLocation(location.getRegionInfo(), anySource, - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() + 1); - location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation(); - Assert.assertEquals(nextPort, location.getPort()); - - // Lower seqNum - does not overwrite higher seqNum. - nextPort = location.getPort() + 1; - conn.updateCachedLocation(location.getRegionInfo(), anySource, - ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1); - location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation(); - Assert.assertEquals(nextPort - 1, location.getPort()); - table.close(); - } - - @Test - public void testClosing() throws Exception { - Configuration configuration = - new Configuration(TEST_UTIL.getConfiguration()); - configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID, - String.valueOf(ThreadLocalRandom.current().nextInt())); - - // as connection caching is going away, now we're just testing - // that closed connection does actually get closed. - - Connection c1 = ConnectionFactory.createConnection(configuration); - Connection c2 = ConnectionFactory.createConnection(configuration); - // no caching, different connections - assertTrue(c1 != c2); - - // closing independently - c1.close(); - assertTrue(c1.isClosed()); - assertFalse(c2.isClosed()); - - c2.close(); - assertTrue(c2.isClosed()); - } - - /** - * Trivial test to verify that nobody messes with - * {@link ConnectionFactory#createConnection(Configuration)} - */ - @Test - public void testCreateConnection() throws Exception { - Configuration configuration = TEST_UTIL.getConfiguration(); - Connection c1 = ConnectionFactory.createConnection(configuration); - Connection c2 = ConnectionFactory.createConnection(configuration); - // created from the same configuration, yet they are different - assertTrue(c1 != c2); - assertTrue(c1.getConfiguration() == c2.getConfiguration()); - } - - /** - * This test checks that one can connect to the cluster with only the - * ZooKeeper quorum set. Other stuff like master address will be read - * from ZK by the client. - */ - @Test - public void testConnection() throws Exception { - // We create an empty config and add the ZK address. - Configuration c = new Configuration(); - c.set(HConstants.ZOOKEEPER_QUORUM, - TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM)); - c.set(HConstants.ZOOKEEPER_CLIENT_PORT, - TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT)); - - // This should be enough to connect - ConnectionImplementation conn = - (ConnectionImplementation) ConnectionFactory.createConnection(c); - assertTrue(conn.isMasterRunning()); - conn.close(); - } - - private int setNumTries(ConnectionImplementation hci, int newVal) throws Exception { - Field numTries = hci.getClass().getDeclaredField("numTries"); - numTries.setAccessible(true); - Field modifiersField = Field.class.getDeclaredField("modifiers"); - modifiersField.setAccessible(true); - modifiersField.setInt(numTries, numTries.getModifiers() & ~Modifier.FINAL); - final int prevNumRetriesVal = (Integer)numTries.get(hci); - numTries.set(hci, newVal); - - return prevNumRetriesVal; - } - - @Test - public void testMulti() throws Exception { - Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM); - try { - ConnectionImplementation conn = (ConnectionImplementation)TEST_UTIL.getConnection(); - - // We're now going to move the region and check that it works for the client - // First a new put to add the location in the cache - conn.clearRegionCache(TABLE_NAME3); - Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME3)); - - TEST_UTIL.getAdmin().balancerSwitch(false, false); - HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster(); - - // We can wait for all regions to be online, that makes log reading easier when debugging - TEST_UTIL.waitUntilNoRegionsInTransition(); - - Put put = new Put(ROW_X); - put.addColumn(FAM_NAM, ROW_X, ROW_X); - table.put(put); - - // Now moving the region to the second server - HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME3, ROW_X).getRegionLocation(); - byte[] regionName = toMove.getRegionInfo().getRegionName(); - byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes(); - - // Choose the other server. - int curServerId = TEST_UTIL.getHBaseCluster().getServerWith(regionName); - int destServerId = (curServerId == 0 ? 1 : 0); - - HRegionServer curServer = TEST_UTIL.getHBaseCluster().getRegionServer(curServerId); - HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(destServerId); - - ServerName destServerName = destServer.getServerName(); - ServerName metaServerName = TEST_UTIL.getHBaseCluster().getServerHoldingMeta(); - - //find another row in the cur server that is less than ROW_X - List regions = curServer.getRegions(TABLE_NAME3); - byte[] otherRow = null; - for (Region region : regions) { - if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName()) - && Bytes.BYTES_COMPARATOR.compare(region.getRegionInfo().getStartKey(), ROW_X) < 0) { - otherRow = region.getRegionInfo().getStartKey(); - break; - } - } - assertNotNull(otherRow); - // If empty row, set it to first row.-f - if (otherRow.length <= 0) otherRow = Bytes.toBytes("aaa"); - Put put2 = new Put(otherRow); - put2.addColumn(FAM_NAM, otherRow, otherRow); - table.put(put2); //cache put2's location - - // Check that we are in the expected state - Assert.assertTrue(curServer != destServer); - Assert.assertNotEquals(curServer.getServerName(), destServer.getServerName()); - Assert.assertNotEquals(toMove.getPort(), destServerName.getPort()); - Assert.assertNotNull(curServer.getOnlineRegion(regionName)); - Assert.assertNull(destServer.getOnlineRegion(regionName)); - Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster(). - getAssignmentManager().hasRegionsInTransition()); - - // Moving. It's possible that we don't have all the regions online at this point, so - // the test depends only on the region we're looking at. - LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString()); - TEST_UTIL.getAdmin().move(toMove.getRegionInfo().getEncodedNameAsBytes(), destServerName); - - while (destServer.getOnlineRegion(regionName) == null || - destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) || - curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) || - master.getAssignmentManager().hasRegionsInTransition()) { - // wait for the move to be finished - Thread.sleep(1); - } - - LOG.info("Move finished for region="+toMove.getRegionInfo().getRegionNameAsString()); - - // Check our new state. - Assert.assertNull(curServer.getOnlineRegion(regionName)); - Assert.assertNotNull(destServer.getOnlineRegion(regionName)); - Assert.assertFalse(destServer.getRegionsInTransitionInRS() - .containsKey(encodedRegionNameBytes)); - Assert.assertFalse(curServer.getRegionsInTransitionInRS() - .containsKey(encodedRegionNameBytes)); - - - // Cache was NOT updated and points to the wrong server - Assert.assertFalse( - conn.getCachedLocation(TABLE_NAME3, ROW_X).getRegionLocation() - .getPort() == destServerName.getPort()); - - // Hijack the number of retry to fail after 2 tries - final int prevNumRetriesVal = setNumTries(conn, 2); - - Put put3 = new Put(ROW_X); - put3.addColumn(FAM_NAM, ROW_X, ROW_X); - Put put4 = new Put(otherRow); - put4.addColumn(FAM_NAM, otherRow, otherRow); - - // do multi - ArrayList actions = Lists.newArrayList(put4, put3); - table.batch(actions, null); // first should be a valid row, - // second we get RegionMovedException. - - setNumTries(conn, prevNumRetriesVal); - } finally { - table.close(); - } - } - - @Test - public void testErrorBackoffTimeCalculation() throws Exception { - // TODO: This test would seem to presume hardcoded RETRY_BACKOFF which it should not. - final long ANY_PAUSE = 100; - ServerName location = ServerName.valueOf("127.0.0.1", 1, 0); - ServerName diffLocation = ServerName.valueOf("127.0.0.1", 2, 0); - - ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge(); - EnvironmentEdgeManager.injectEdge(timeMachine); - try { - long largeAmountOfTime = ANY_PAUSE * 1000; - ConnectionImplementation.ServerErrorTracker tracker = - new ConnectionImplementation.ServerErrorTracker(largeAmountOfTime, 100); - - // The default backoff is 0. - assertEquals(0, tracker.calculateBackoffTime(location, ANY_PAUSE)); - - // Check some backoff values from HConstants sequence. - tracker.reportServerError(location); - assertEqualsWithJitter(ANY_PAUSE * HConstants.RETRY_BACKOFF[0], - tracker.calculateBackoffTime(location, ANY_PAUSE)); - tracker.reportServerError(location); - tracker.reportServerError(location); - tracker.reportServerError(location); - assertEqualsWithJitter(ANY_PAUSE * HConstants.RETRY_BACKOFF[3], - tracker.calculateBackoffTime(location, ANY_PAUSE)); - - // All of this shouldn't affect backoff for different location. - assertEquals(0, tracker.calculateBackoffTime(diffLocation, ANY_PAUSE)); - tracker.reportServerError(diffLocation); - assertEqualsWithJitter(ANY_PAUSE * HConstants.RETRY_BACKOFF[0], - tracker.calculateBackoffTime(diffLocation, ANY_PAUSE)); - - // Check with different base. - assertEqualsWithJitter(ANY_PAUSE * 2 * HConstants.RETRY_BACKOFF[3], - tracker.calculateBackoffTime(location, ANY_PAUSE * 2)); - } finally { - EnvironmentEdgeManager.reset(); - } - } - - private static void assertEqualsWithJitter(long expected, long actual) { - assertEqualsWithJitter(expected, actual, expected); - } - - private static void assertEqualsWithJitter(long expected, long actual, long jitterBase) { - assertTrue("Value not within jitter: " + expected + " vs " + actual, - Math.abs(actual - expected) <= (0.01f * jitterBase)); - } - - @Test - public void testConnectionRideOverClusterRestart() throws IOException, InterruptedException { - Configuration config = new Configuration(TEST_UTIL.getConfiguration()); - - final TableName tableName = TableName.valueOf(name.getMethodName()); - TEST_UTIL.createTable(tableName, new byte[][] {FAM_NAM}).close(); - - Connection connection = ConnectionFactory.createConnection(config); - Table table = connection.getTable(tableName); - - // this will cache the meta location and table's region location - table.get(new Get(Bytes.toBytes("foo"))); - - // restart HBase - TEST_UTIL.shutdownMiniHBaseCluster(); - TEST_UTIL.restartHBaseCluster(2); - // this should be able to discover new locations for meta and table's region - table.get(new Get(Bytes.toBytes("foo"))); - TEST_UTIL.deleteTable(tableName); - table.close(); - connection.close(); - } - - @Test - public void testLocateRegionsWithRegionReplicas() throws IOException { - int regionReplication = 3; - byte[] family = Bytes.toBytes("cf"); - TableName tableName = TableName.valueOf(name.getMethodName()); - - // Create a table with region replicas - TableDescriptorBuilder builder = TableDescriptorBuilder - .newBuilder(tableName) - .setRegionReplication(regionReplication) - .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)); - TEST_UTIL.getAdmin().createTable(builder.build()); - - try (ConnectionImplementation con = (ConnectionImplementation) ConnectionFactory. - createConnection(TEST_UTIL.getConfiguration())) { - // Get locations of the regions of the table - List locations = con.locateRegions(tableName, false, false); - - // The size of the returned locations should be 3 - assertEquals(regionReplication, locations.size()); - - // The replicaIds of the returned locations should be 0, 1 and 2 - Set expectedReplicaIds = IntStream.range(0, regionReplication). - boxed().collect(Collectors.toSet()); - for (HRegionLocation location : locations) { - assertTrue(expectedReplicaIds.remove(location.getRegion().getReplicaId())); - } - } finally { - TEST_UTIL.deleteTable(tableName); - } - } - - @Test - public void testMetaLookupThreadPoolCreated() throws Exception { - final TableName tableName = TableName.valueOf(name.getMethodName()); - byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") }; - if (TEST_UTIL.getAdmin().tableExists(tableName)) { - TEST_UTIL.getAdmin().disableTable(tableName); - TEST_UTIL.getAdmin().deleteTable(tableName); - } - try (Table htable = TEST_UTIL.createTable(tableName, FAMILIES)) { - byte[] row = Bytes.toBytes("test"); - ConnectionImplementation c = ((ConnectionImplementation) TEST_UTIL.getConnection()); - // check that metalookup pool would get created - c.relocateRegion(tableName, row); - ExecutorService ex = c.getCurrentMetaLookupPool(); - assertNotNull(ex); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java deleted file mode 100644 index 9e65c57f4833..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java +++ /dev/null @@ -1,387 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.exceptions.PreemptiveFastFailException; -import org.apache.hadoop.hbase.ipc.RpcExecutor; -import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler; -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.LoadTestKVGenerator; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category({MediumTests.class, ClientTests.class}) -public class TestFastFail { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestFastFail.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestFastFail.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static byte[] FAMILY = Bytes.toBytes("testFamily"); - private static final Random random = new Random(); - private static int SLAVES = 1; - private static byte[] QUALIFIER = Bytes.toBytes("testQualifier"); - private static final int SLEEPTIME = 5000; - - @Rule - public TestName name = new TestName(); - - /** - * @throws java.lang.Exception - */ - @BeforeClass - public static void setUpBeforeClass() throws Exception { - // Just to prevent fastpath FIFO from picking calls up bypassing the queue. - TEST_UTIL.getConfiguration().set( - RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY, "deadline"); - TEST_UTIL.startMiniCluster(SLAVES); - } - - /** - * @throws java.lang.Exception - */ - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - /** - * @throws java.lang.Exception - */ - @Before - public void setUp() throws Exception { - MyPreemptiveFastFailInterceptor.numBraveSouls.set(0); - CallQueueTooBigPffeInterceptor.numCallQueueTooBig.set(0); - } - - /** - * @throws java.lang.Exception - */ - @After - public void tearDown() throws Exception { - // Nothing to do. - } - - @Ignore ("Can go zombie -- see HBASE-14421; FIX") @Test - public void testFastFail() throws IOException, InterruptedException { - Admin admin = TEST_UTIL.getAdmin(); - - final String tableName = name.getMethodName(); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes - .toBytes(tableName))); - desc.addFamily(new HColumnDescriptor(FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaaa"), Bytes.toBytes("zzzz"), 32); - final long numRows = 1000; - - Configuration conf = TEST_UTIL.getConfiguration(); - conf.setLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, SLEEPTIME * 100); - conf.setInt(HConstants.HBASE_CLIENT_PAUSE, SLEEPTIME / 10); - conf.setBoolean(HConstants.HBASE_CLIENT_FAST_FAIL_MODE_ENABLED, true); - conf.setLong(HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS, 0); - conf.setClass(HConstants.HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL, - MyPreemptiveFastFailInterceptor.class, - PreemptiveFastFailInterceptor.class); - - final Connection connection = ConnectionFactory.createConnection(conf); - - /** - * Write numRows worth of data, so that the workers can arbitrarily read. - */ - List puts = new ArrayList<>(); - for (long i = 0; i < numRows; i++) { - byte[] rowKey = longToByteArrayKey(i); - Put put = new Put(rowKey); - byte[] value = rowKey; // value is the same as the row key - put.addColumn(FAMILY, QUALIFIER, value); - puts.add(put); - } - try (Table table = connection.getTable(TableName.valueOf(tableName))) { - table.put(puts); - LOG.info("Written all puts."); - } - - /** - * The number of threads that are going to perform actions against the test - * table. - */ - int nThreads = 100; - ExecutorService service = Executors.newFixedThreadPool(nThreads); - final CountDownLatch continueOtherHalf = new CountDownLatch(1); - final CountDownLatch doneHalfway = new CountDownLatch(nThreads); - - final AtomicInteger numSuccessfullThreads = new AtomicInteger(0); - final AtomicInteger numFailedThreads = new AtomicInteger(0); - - // The total time taken for the threads to perform the second put; - final AtomicLong totalTimeTaken = new AtomicLong(0); - final AtomicInteger numBlockedWorkers = new AtomicInteger(0); - final AtomicInteger numPreemptiveFastFailExceptions = new AtomicInteger(0); - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - futures.add(service.submit(new Callable() { - /** - * The workers are going to perform a couple of reads. The second read - * will follow the killing of a regionserver so that we make sure that - * some of threads go into PreemptiveFastFailExcception - */ - @Override - public Boolean call() throws Exception { - try (Table table = connection.getTable(TableName.valueOf(tableName))) { - Thread.sleep(Math.abs(random.nextInt()) % 250); // Add some jitter here - byte[] row = longToByteArrayKey(Math.abs(random.nextLong()) - % numRows); - Get g = new Get(row); - g.addColumn(FAMILY, QUALIFIER); - try { - table.get(g); - } catch (Exception e) { - LOG.debug("Get failed : ", e); - doneHalfway.countDown(); - return false; - } - - // Done with one get, proceeding to do the next one. - doneHalfway.countDown(); - continueOtherHalf.await(); - - long startTime = System.currentTimeMillis(); - g = new Get(row); - g.addColumn(FAMILY, QUALIFIER); - try { - table.get(g); - // The get was successful - numSuccessfullThreads.addAndGet(1); - } catch (Exception e) { - if (e instanceof PreemptiveFastFailException) { - // We were issued a PreemptiveFastFailException - numPreemptiveFastFailExceptions.addAndGet(1); - } - // Irrespective of PFFE, the request failed. - numFailedThreads.addAndGet(1); - return false; - } finally { - long enTime = System.currentTimeMillis(); - totalTimeTaken.addAndGet(enTime - startTime); - if ((enTime - startTime) >= SLEEPTIME) { - // Considering the slow workers as the blockedWorkers. - // This assumes that the threads go full throttle at performing - // actions. In case the thread scheduling itself is as slow as - // SLEEPTIME, then this test might fail and so, we might have - // set it to a higher number on slower machines. - numBlockedWorkers.addAndGet(1); - } - } - return true; - } catch (Exception e) { - LOG.error("Caught unknown exception", e); - doneHalfway.countDown(); - return false; - } - } - })); - } - - doneHalfway.await(); - - // Kill a regionserver - TEST_UTIL.getHBaseCluster().getRegionServer(0).getRpcServer().stop(); - TEST_UTIL.getHBaseCluster().getRegionServer(0).stop("Testing"); - - // Let the threads continue going - continueOtherHalf.countDown(); - - Thread.sleep(2 * SLEEPTIME); - // Start a RS in the cluster - TEST_UTIL.getHBaseCluster().startRegionServer(); - - int numThreadsReturnedFalse = 0; - int numThreadsReturnedTrue = 0; - int numThreadsThrewExceptions = 0; - for (Future f : futures) { - try { - numThreadsReturnedTrue += f.get() ? 1 : 0; - numThreadsReturnedFalse += f.get() ? 0 : 1; - } catch (Exception e) { - numThreadsThrewExceptions++; - } - } - LOG.debug("numThreadsReturnedFalse:" - + numThreadsReturnedFalse - + " numThreadsReturnedTrue:" - + numThreadsReturnedTrue - + " numThreadsThrewExceptions:" - + numThreadsThrewExceptions - + " numFailedThreads:" - + numFailedThreads.get() - + " numSuccessfullThreads:" - + numSuccessfullThreads.get() - + " numBlockedWorkers:" - + numBlockedWorkers.get() - + " totalTimeWaited: " - + totalTimeTaken.get() - / (numBlockedWorkers.get() == 0 ? Long.MAX_VALUE : numBlockedWorkers - .get()) + " numPFFEs: " + numPreemptiveFastFailExceptions.get()); - - assertEquals("The expected number of all the successfull and the failed " - + "threads should equal the total number of threads that we spawned", - nThreads, numFailedThreads.get() + numSuccessfullThreads.get()); - assertEquals( - "All the failures should be coming from the secondput failure", - numFailedThreads.get(), numThreadsReturnedFalse); - assertEquals("Number of threads that threw execution exceptions " - + "otherwise should be 0", 0, numThreadsThrewExceptions); - assertEquals("The regionservers that returned true should equal to the" - + " number of successful threads", numThreadsReturnedTrue, - numSuccessfullThreads.get()); - assertTrue( - "There will be atleast one thread that retried instead of failing", - MyPreemptiveFastFailInterceptor.numBraveSouls.get() > 0); - assertTrue( - "There will be atleast one PreemptiveFastFail exception," - + " otherwise, the test makes little sense." - + "numPreemptiveFastFailExceptions: " - + numPreemptiveFastFailExceptions.get(), - numPreemptiveFastFailExceptions.get() > 0); - - assertTrue( - "Only few thread should ideally be waiting for the dead " - + "regionserver to be coming back. numBlockedWorkers:" - + numBlockedWorkers.get() + " threads that retried : " - + MyPreemptiveFastFailInterceptor.numBraveSouls.get(), - numBlockedWorkers.get() <= MyPreemptiveFastFailInterceptor.numBraveSouls - .get()); - } - - @Test - public void testCallQueueTooBigExceptionDoesntTriggerPffe() throws Exception { - Admin admin = TEST_UTIL.getAdmin(); - - final String tableName = name.getMethodName(); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes - .toBytes(tableName))); - desc.addFamily(new HColumnDescriptor(FAMILY)); - admin.createTable(desc, Bytes.toBytes("aaaa"), Bytes.toBytes("zzzz"), 3); - - Configuration conf = TEST_UTIL.getConfiguration(); - conf.setLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 100); - conf.setInt(HConstants.HBASE_CLIENT_PAUSE, 500); - conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - - conf.setBoolean(HConstants.HBASE_CLIENT_FAST_FAIL_MODE_ENABLED, true); - conf.setLong(HConstants.HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS, 0); - conf.setClass(HConstants.HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL, - CallQueueTooBigPffeInterceptor.class, PreemptiveFastFailInterceptor.class); - - final Connection connection = ConnectionFactory.createConnection(conf); - - //Set max call queues size to 0 - SimpleRpcScheduler srs = (SimpleRpcScheduler) - TEST_UTIL.getHBaseCluster().getRegionServer(0).getRpcServer().getScheduler(); - Configuration newConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); - newConf.setInt("hbase.ipc.server.max.callqueue.length", 0); - srs.onConfigurationChange(newConf); - - try (Table table = connection.getTable(TableName.valueOf(tableName))) { - Get get = new Get(new byte[1]); - table.get(get); - } catch (Throwable ex) { - } - - assertEquals("We should have not entered PFFE mode on CQTBE, but we did;" - + " number of times this mode should have been entered:", 0, - CallQueueTooBigPffeInterceptor.numCallQueueTooBig.get()); - - newConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); - newConf.setInt("hbase.ipc.server.max.callqueue.length", 250); - srs.onConfigurationChange(newConf); - } - - public static class MyPreemptiveFastFailInterceptor extends - PreemptiveFastFailInterceptor { - public static AtomicInteger numBraveSouls = new AtomicInteger(); - - @Override - protected boolean shouldRetryInspiteOfFastFail(FailureInfo fInfo) { - boolean ret = super.shouldRetryInspiteOfFastFail(fInfo); - if (ret) - numBraveSouls.addAndGet(1); - return ret; - } - - public MyPreemptiveFastFailInterceptor(Configuration conf) { - super(conf); - } - } - - private byte[] longToByteArrayKey(long rowKey) { - return Bytes.toBytes(LoadTestKVGenerator.md5PrefixedKey(rowKey)); - } - - public static class CallQueueTooBigPffeInterceptor extends - PreemptiveFastFailInterceptor { - public static AtomicInteger numCallQueueTooBig = new AtomicInteger(); - - @Override - protected void handleFailureToServer(ServerName serverName, Throwable t) { - super.handleFailureToServer(serverName, t); - numCallQueueTooBig.incrementAndGet(); - } - - public CallQueueTooBigPffeInterceptor(Configuration conf) { - super(conf); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index ea16ef5f6298..48597298742e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -174,8 +174,8 @@ public static void tearDownAfterClass() throws Exception { public void testDuplicateAppend() throws Exception { HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(name.getMethodName()); Map kvs = new HashMap<>(); - kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000"); - hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs); + kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000"); + hdt.addCoprocessor(SleepAtFirstRpcCall.class.getName(), null, 1, kvs); TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close(); Configuration c = new Configuration(TEST_UTIL.getConfiguration()); @@ -6317,31 +6317,6 @@ public void testFilterAllRecords() throws IOException { } } - // to be removed - @Ignore - @Test - public void testRegionCache() throws IOException { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - HColumnDescriptor fam = new HColumnDescriptor(FAMILY); - htd.addFamily(fam); - byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE; - Admin admin = TEST_UTIL.getAdmin(); - admin.createTable(htd, KEYS); - HRegionLocator locator = - (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName()); - List results = locator.getAllRegionLocations(); - int number = ((ConnectionImplementation)admin.getConnection()) - .getNumberOfCachedRegionLocations(htd.getTableName()); - assertEquals(results.size(), number); - ConnectionImplementation conn = ((ConnectionImplementation)admin.getConnection()); - assertNotNull("Can't get cached location for row aaa", - conn.getCachedLocation(htd.getTableName(),Bytes.toBytes("aaa"))); - for(byte[] startKey:HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE){ - assertNotNull("Can't get cached location for row "+ - Bytes.toString(startKey),(conn.getCachedLocation(htd.getTableName(),startKey))); - } - } - @Test public void testCellSizeLimit() throws IOException { final TableName tableName = TableName.valueOf("testCellSizeLimit"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java deleted file mode 100644 index c00f4b65fac9..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ /dev/null @@ -1,336 +0,0 @@ -/** - * 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 static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import com.google.protobuf.ServiceException; -import java.io.IOException; -import java.util.ArrayList; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.PleaseHoldException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; -import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestName; -import org.mockito.Matchers; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest; - -@Category({SmallTests.class, ClientTests.class}) -public class TestHBaseAdminNoCluster { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestHBaseAdminNoCluster.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestHBaseAdminNoCluster.class); - - @Rule - public TestName name = new TestName(); - - /** - * Verify that PleaseHoldException gets retried. - * HBASE-8764 - */ - //TODO: Clean up, with Procedure V2 and nonce to prevent the same procedure to call mulitple - // time, this test is invalid anymore. Just keep the test around for some time before - // fully removing it. - @Ignore - @Test - public void testMasterMonitorCallableRetries() - throws MasterNotRunningException, ZooKeeperConnectionException, IOException, - org.apache.hbase.thirdparty.com.google.protobuf.ServiceException { - Configuration configuration = HBaseConfiguration.create(); - // Set the pause and retry count way down. - configuration.setLong(HConstants.HBASE_CLIENT_PAUSE, 1); - final int count = 10; - configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, count); - // Get mocked connection. Getting the connection will register it so when HBaseAdmin is - // constructed with same configuration, it will find this mocked connection. - ConnectionImplementation connection = - HConnectionTestingUtility.getMockedConnection(configuration); - // Mock so we get back the master interface. Make it so when createTable is called, we throw - // the PleaseHoldException. - MasterKeepAliveConnection masterAdmin = Mockito.mock(MasterKeepAliveConnection.class); - Mockito.when(masterAdmin.createTable((RpcController)Mockito.any(), - (CreateTableRequest)Mockito.any())). - thenThrow(new ServiceException("Test fail").initCause(new PleaseHoldException("test"))); - Mockito.when(connection.getMaster()).thenReturn(masterAdmin); - Admin admin = new HBaseAdmin(connection); - try { - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - // Pass any old htable descriptor; not important - try { - admin.createTable(htd, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); - fail(); - } catch (RetriesExhaustedException e) { - LOG.info("Expected fail", e); - } - // Assert we were called 'count' times. - Mockito.verify(masterAdmin, Mockito.atLeast(count)).createTable((RpcController)Mockito.any(), - (CreateTableRequest)Mockito.any()); - } finally { - admin.close(); - if (connection != null) connection.close(); - } - } - - @Test - public void testMasterOperationsRetries() throws Exception { - - // Admin.listTables() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.listTableDescriptors(); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .getTableDescriptors((RpcController)Mockito.any(), - (GetTableDescriptorsRequest)Mockito.any()); - } - }); - - // Admin.listTableNames() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.listTableNames(); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .getTableNames((RpcController)Mockito.any(), - (GetTableNamesRequest)Mockito.any()); - } - }); - - // Admin.getDescriptor() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.getDescriptor(TableName.valueOf(name.getMethodName())); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .getTableDescriptors((RpcController)Mockito.any(), - (GetTableDescriptorsRequest)Mockito.any()); - } - }); - - // Admin.getTableDescriptorsByTableName() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.listTableDescriptors(new ArrayList<>()); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .getTableDescriptors((RpcController)Mockito.any(), - (GetTableDescriptorsRequest)Mockito.any()); - } - }); - - // Admin.move() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.move(new byte[0]); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .moveRegion((RpcController)Mockito.any(), - (MoveRegionRequest)Mockito.any()); - } - }); - - // Admin.offline() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.offline(new byte[0]); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .offlineRegion((RpcController)Mockito.any(), - (OfflineRegionRequest)Mockito.any()); - } - }); - - // Admin.balancerSwitch() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.balancerSwitch(true, true); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .setBalancerRunning((RpcController)Mockito.any(), - (SetBalancerRunningRequest)Mockito.any()); - } - }); - - // Admin.balance() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.balance(); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .balance((RpcController)Mockito.any(), - (BalanceRequest)Mockito.any()); - } - }); - - // Admin.enabledCatalogJanitor() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.catalogJanitorSwitch(true); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .enableCatalogJanitor((RpcController)Mockito.any(), - (EnableCatalogJanitorRequest)Mockito.any()); - } - }); - - // Admin.runCatalogScan() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.runCatalogJanitor(); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .runCatalogScan((RpcController)Mockito.any(), - (RunCatalogScanRequest)Mockito.any()); - } - }); - - // Admin.isCatalogJanitorEnabled() - testMasterOperationIsRetried(new MethodCaller() { - @Override - public void call(Admin admin) throws Exception { - admin.isCatalogJanitorEnabled(); - } - @Override - public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception { - Mockito.verify(masterAdmin, Mockito.atLeast(count)) - .isCatalogJanitorEnabled((RpcController)Mockito.any(), - (IsCatalogJanitorEnabledRequest)Mockito.any()); - } - }); - } - - private static interface MethodCaller { - void call(Admin admin) throws Exception; - void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception; - } - - private void testMasterOperationIsRetried(MethodCaller caller) throws Exception { - Configuration configuration = HBaseConfiguration.create(); - // Set the pause and retry count way down. - configuration.setLong(HConstants.HBASE_CLIENT_PAUSE, 1); - final int count = 10; - configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, count); - - ConnectionImplementation connection = mock(ConnectionImplementation.class); - when(connection.getConfiguration()).thenReturn(configuration); - MasterKeepAliveConnection masterAdmin = - Mockito.mock(MasterKeepAliveConnection.class, new Answer() { - @Override - public Object answer(InvocationOnMock invocation) throws Throwable { - if (invocation.getMethod().getName().equals("close")) { - return null; - } - throw new MasterNotRunningException(); // all methods will throw an exception - } - }); - Mockito.when(connection.getMaster()).thenReturn(masterAdmin); - RpcControllerFactory rpcControllerFactory = Mockito.mock(RpcControllerFactory.class); - Mockito.when(connection.getRpcControllerFactory()).thenReturn(rpcControllerFactory); - Mockito.when(rpcControllerFactory.newController()).thenReturn( - Mockito.mock(HBaseRpcController.class)); - - // we need a real retrying caller - RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(configuration); - Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); - - Admin admin = null; - try { - admin = Mockito.spy(new HBaseAdmin(connection)); - // mock the call to getRegion since in the absence of a cluster (which means the meta - // is not assigned), getRegion can't function - Mockito.doReturn(null).when(((HBaseAdmin)admin)).getRegion(Matchers.any()); - try { - caller.call(admin); // invoke the HBaseAdmin method - fail(); - } catch (RetriesExhaustedException e) { - LOG.info("Expected fail", e); - } - // Assert we were called 'count' times. - caller.verify(masterAdmin, count); - } finally { - if (admin != null) {admin.close();} - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java index 58b30e4ce70a..45afd1cf70c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java @@ -103,8 +103,8 @@ public static void afterClass() throws Exception { public void testDuplicateIncrement() throws Exception { HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName())); Map kvs = new HashMap<>(); - kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000"); - hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs); + kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000"); + hdt.addCoprocessor(SleepAtFirstRpcCall.class.getName(), null, 1, kvs); TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close(); Configuration c = new Configuration(TEST_UTIL.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index bb58930a5aae..4bfdcad029ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -17,22 +17,20 @@ */ package org.apache.hadoop.hbase.client; -import static junit.framework.Assert.assertEquals; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; -import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CallQueueTooBigException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; @@ -46,13 +44,14 @@ import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; @@ -60,10 +59,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -/** - * Will be removed along with ConnectionImplementation soon. - */ -@Ignore @Category({MediumTests.class, ClientTests.class}) public class TestMetaCache { @@ -78,45 +73,51 @@ public class TestMetaCache { private static HRegionServer badRS; - /** - * @throws java.lang.Exception - */ + private Connection conn; + private MetricsConnection metrics; + private AsyncRegionLocator locator; + @BeforeClass public static void setUpBeforeClass() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); - conf.setStrings(HConstants.REGION_SERVER_IMPL, - RegionServerWithFakeRpcServices.class.getName()); + conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithFakeRpcServices.class.getName()); TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME.META_TABLE_NAME); + TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); badRS = TEST_UTIL.getHBaseCluster().getRegionServer(0); assertTrue(badRS.getRSRpcServices() instanceof FakeRSRpcServices); - HTableDescriptor table = new HTableDescriptor(TABLE_NAME); - HColumnDescriptor fam = new HColumnDescriptor(FAMILY); - fam.setMaxVersions(2); - table.addFamily(fam); - TEST_UTIL.createTable(table, null); + TableDescriptor desc = TableDescriptorBuilder.newBuilder(TABLE_NAME) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(2).build()) + .build(); + TEST_UTIL.createTable(desc, null); } - - /** - * @throws java.lang.Exception - */ @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); } + @After + public void tearDown() throws IOException { + Closeables.close(conn, true); + } + + private void setupConnection(int retry) throws IOException { + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retry); + conf.setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true); + conn = ConnectionFactory.createConnection(conf); + AsyncConnectionImpl asyncConn = (AsyncConnectionImpl) conn.toAsyncConnection(); + locator = asyncConn.getLocator(); + metrics = asyncConn.getConnectionMetrics().get(); + } + @Test public void testPreserveMetaCacheOnException() throws Exception { - ((FakeRSRpcServices)badRS.getRSRpcServices()).setExceptionInjector( - new RoundRobinExceptionInjector()); - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - conf.set("hbase.client.retries.number", "1"); - ConnectionImplementation conn = - (ConnectionImplementation) ConnectionFactory.createConnection(conf); - try { - Table table = conn.getTable(TABLE_NAME); + ((FakeRSRpcServices) badRS.getRSRpcServices()) + .setExceptionInjector(new RoundRobinExceptionInjector()); + setupConnection(1); + try (Table table = conn.getTable(TABLE_NAME)){ byte[] row = Bytes.toBytes("row1"); Put put = new Put(row); @@ -154,66 +155,51 @@ public void testPreserveMetaCacheOnException() throws Exception { } // Do not test if we did not touch the meta cache in this iteration. if (exp != null && ClientExceptionsUtil.isMetaClearingException(exp)) { - assertNull(conn.getCachedLocation(TABLE_NAME, row)); + assertNull(locator.getRegionLocationInCache(TABLE_NAME, row)); } else if (success) { - assertNotNull(conn.getCachedLocation(TABLE_NAME, row)); + assertNotNull(locator.getRegionLocationInCache(TABLE_NAME, row)); } } - } finally { - conn.close(); } } @Test public void testCacheClearingOnCallQueueTooBig() throws Exception { - ((FakeRSRpcServices)badRS.getRSRpcServices()).setExceptionInjector( - new CallQueueTooBigExceptionInjector()); - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - conf.set("hbase.client.retries.number", "2"); - conf.set(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, "true"); - ConnectionImplementation conn = - (ConnectionImplementation) ConnectionFactory.createConnection(conf); + ((FakeRSRpcServices) badRS.getRSRpcServices()) + .setExceptionInjector(new CallQueueTooBigExceptionInjector()); + setupConnection(2); + Table table = conn.getTable(TABLE_NAME); + byte[] row = Bytes.toBytes("row1"); + + Put put = new Put(row); + put.addColumn(FAMILY, QUALIFIER, Bytes.toBytes(10)); + table.put(put); + + // obtain the client metrics + long preGetRegionClears = metrics.metaCacheNumClearRegion.getCount(); + long preGetServerClears = metrics.metaCacheNumClearServer.getCount(); + + // attempt a get on the test table + Get get = new Get(row); try { - Table table = conn.getTable(TABLE_NAME); - byte[] row = Bytes.toBytes("row1"); - - Put put = new Put(row); - put.addColumn(FAMILY, QUALIFIER, Bytes.toBytes(10)); - table.put(put); - - // obtain the client metrics - MetricsConnection metrics = conn.getConnectionMetrics(); - long preGetRegionClears = metrics.metaCacheNumClearRegion.getCount(); - long preGetServerClears = metrics.metaCacheNumClearServer.getCount(); - - // attempt a get on the test table - Get get = new Get(row); - try { - table.get(get); - fail("Expected CallQueueTooBigException"); - } catch (RetriesExhaustedException ree) { - // expected - } - - // verify that no cache clearing took place - long postGetRegionClears = metrics.metaCacheNumClearRegion.getCount(); - long postGetServerClears = metrics.metaCacheNumClearServer.getCount(); - assertEquals(preGetRegionClears, postGetRegionClears); - assertEquals(preGetServerClears, postGetServerClears); - } finally { - conn.close(); + table.get(get); + fail("Expected CallQueueTooBigException"); + } catch (RetriesExhaustedException ree) { + // expected } + + // verify that no cache clearing took place + long postGetRegionClears = metrics.metaCacheNumClearRegion.getCount(); + long postGetServerClears = metrics.metaCacheNumClearServer.getCount(); + assertEquals(preGetRegionClears, postGetRegionClears); + assertEquals(preGetServerClears, postGetServerClears); } public static List metaCachePreservingExceptions() { - return new ArrayList() {{ - add(new RegionOpeningException(" ")); - add(new RegionTooBusyException("Some old message")); - add(new RpcThrottlingException(" ")); - add(new MultiActionResultTooLarge(" ")); - add(new RetryImmediatelyException(" ")); - add(new CallQueueTooBigException()); - }}; + return Arrays.asList(new RegionOpeningException(" "), + new RegionTooBusyException("Some old message"), new RpcThrottlingException(" "), + new MultiActionResultTooLarge(" "), new RetryImmediatelyException(" "), + new CallQueueTooBigException()); } public static class RegionServerWithFakeRpcServices extends HRegionServer { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java index 108ab7f3f4dd..dd1965d3c5b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -25,42 +24,19 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.NavigableMap; -import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellScannable; -import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; -import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse; /** * Test MetaTableAccessor but without spinning up a cluster. @@ -73,21 +49,7 @@ public class TestMetaTableAccessorNoCluster { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMetaTableAccessorNoCluster.class); - private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableAccessorNoCluster.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - private static final Abortable ABORTABLE = new Abortable() { - boolean aborted = false; - @Override - public void abort(String why, Throwable e) { - LOG.info(why, e); - this.aborted = true; - throw new RuntimeException(e); - } - @Override - public boolean isAborted() { - return this.aborted; - } - }; @Before public void before() throws Exception { @@ -124,98 +86,4 @@ public void testGetHRegionInfo() throws IOException { assertNotNull(hri); assertTrue(RegionInfo.COMPARATOR.compare(hri, RegionInfoBuilder.FIRST_META_REGIONINFO) == 0); } - - /** - * Test that MetaTableAccessor will ride over server throwing - * "Server not running" IOEs. - * @see HBASE-3446 - */ - @Test - public void testRideOverServerNotRunning() - throws IOException, InterruptedException, ServiceException { - // Need a zk watcher. - ZKWatcher zkw = new ZKWatcher(UTIL.getConfiguration(), - this.getClass().getSimpleName(), ABORTABLE, true); - // This is a servername we use in a few places below. - ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis()); - - ConnectionImplementation connection = null; - try { - // Mock an ClientProtocol. Our mock implementation will fail a few - // times when we go to open a scanner. - final ClientProtos.ClientService.BlockingInterface implementation = - Mockito.mock(ClientProtos.ClientService.BlockingInterface.class); - // When scan called throw IOE 'Server not running' a few times - // before we return a scanner id. Whats WEIRD is that these - // exceptions do not show in the log because they are caught and only - // printed if we FAIL. We eventually succeed after retry so these don't - // show. We will know if they happened or not because we will ask - // mockito at the end of this test to verify that scan was indeed - // called the wanted number of times. - List kvs = new ArrayList<>(); - final byte [] rowToVerify = Bytes.toBytes("rowToVerify"); - kvs.add(new KeyValue(rowToVerify, - HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, - RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO))); - kvs.add(new KeyValue(rowToVerify, - HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, - Bytes.toBytes(sn.getAddress().toString()))); - kvs.add(new KeyValue(rowToVerify, - HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, - Bytes.toBytes(sn.getStartcode()))); - final List cellScannables = new ArrayList<>(1); - cellScannables.add(Result.create(kvs)); - final ScanResponse.Builder builder = ScanResponse.newBuilder(); - for (CellScannable result : cellScannables) { - builder.addCellsPerResult(((Result)result).size()); - } - Mockito.when(implementation.scan((RpcController) Mockito.any(), (ScanRequest) Mockito.any())) - .thenThrow(new ServiceException("Server not running (1 of 3)")) - .thenThrow(new ServiceException("Server not running (2 of 3)")) - .thenThrow(new ServiceException("Server not running (3 of 3)")) - .thenAnswer(new Answer() { - @Override - public ScanResponse answer(InvocationOnMock invocation) throws Throwable { - ((HBaseRpcController) invocation.getArgument(0)).setCellScanner(CellUtil - .createCellScanner(cellScannables)); - return builder.setScannerId(1234567890L).setMoreResults(false).build(); - } - }); - // Associate a spied-upon Connection with UTIL.getConfiguration. Need - // to shove this in here first so it gets picked up all over; e.g. by - // HTable. - connection = HConnectionTestingUtility.getSpiedConnection(UTIL.getConfiguration()); - - // Fix the location lookup so it 'works' though no network. First - // make an 'any location' object. - final HRegionLocation anyLocation = - new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, sn); - final RegionLocations rl = new RegionLocations(anyLocation); - // Return the RegionLocations object when locateRegion - // The ugly format below comes of 'Important gotcha on spying real objects!' from - // http://mockito.googlecode.com/svn/branches/1.6/javadoc/org/mockito/Mockito.html - Mockito.doReturn(rl).when(connection).locateRegion((TableName) Mockito.any(), - (byte[]) Mockito.any(), Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyInt()); - - // Now shove our HRI implementation into the spied-upon connection. - Mockito.doReturn(implementation).when(connection).getClient(Mockito.any()); - - // Scan meta for user tables and verify we got back expected answer. - NavigableMap hris = - MetaTableAccessor.getServerUserRegions(connection, sn); - assertEquals(1, hris.size()); - assertTrue(RegionInfo.COMPARATOR.compare(hris.firstEntry().getKey(), - RegionInfoBuilder.FIRST_META_REGIONINFO) == 0); - assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow())); - // Finally verify that scan was called four times -- three times - // with exception and then on 4th attempt we succeed - Mockito.verify(implementation, Mockito.times(4)).scan((RpcController) Mockito.any(), - (ScanRequest) Mockito.any()); - } finally { - if (connection != null && !connection.isClosed()) { - connection.close(); - } - zkw.close(); - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java index 53b44eeb2ca4..55fc289dca91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; @@ -207,9 +206,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtility util) throws } byte[] row = Bytes.toBytes("test"); ServerName master = null; - try ( - ConnectionImplementation c = ConnectionFactory.createConnectionImpl(util.getConfiguration(), - null, UserProvider.instantiate(util.getConfiguration()).getCurrent())) { + try (Connection c = ConnectionFactory.createConnection(util.getConfiguration())) { try (Table htable = util.createTable(TABLE, FAMILIES)) { util.getAdmin().flush(TableName.META_TABLE_NAME); Thread.sleep( @@ -339,8 +336,7 @@ public void testShutdownOfReplicaHolder() throws Exception { // checks that the when the server holding meta replica is shut down, the meta replica // can be recovered try ( - Connection conn = ConnectionFactory.createConnectionImpl(TEST_UTIL.getConfiguration(), null, - UserProvider.instantiate(TEST_UTIL.getConfiguration()).getCurrent()); + Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); RegionLocator locator = conn.getRegionLocator(TableName.META_TABLE_NAME)) { HRegionLocation hrl = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1); ServerName oldServer = hrl.getServerName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java index 9db29cea7fd0..5775ac077922 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -63,8 +62,7 @@ public static void tearDownAfterClass() throws Exception { public void testMultiMetrics() throws Exception { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.set(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, "true"); - try (ConnectionImplementation conn = ConnectionFactory.createConnectionImpl(conf, null, - UserProvider.instantiate(conf).getCurrent())) { + try (Connection conn = ConnectionFactory.createConnection(conf)) { BufferedMutator mutator = conn.getBufferedMutator(TABLE_NAME); byte[][] keys = { Bytes.toBytes("aaa"), Bytes.toBytes("mmm"), Bytes.toBytes("zzz") }; for (byte[] key : keys) { @@ -76,7 +74,8 @@ public void testMultiMetrics() throws Exception { mutator.flush(); mutator.close(); - MetricsConnection metrics = conn.getConnectionMetrics(); + MetricsConnection metrics = + ((AsyncConnectionImpl) conn.toAsyncConnection()).getConnectionMetrics().get(); assertEquals(1, metrics.multiTracker.reqHist.getCount()); assertEquals(3, metrics.numActionsPerServerHist.getSnapshot().getMean(), 1e-15); assertEquals(1, metrics.numActionsPerServerHist.getCount()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 73270d4bd0c0..487a84916f6c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.Cell; @@ -43,12 +42,10 @@ import org.apache.hadoop.hbase.coprocessor.MasterObserver; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.master.RegionPlan; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; -import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -480,106 +477,6 @@ public void testBatchWithIncrementAndAppend() throws Exception { table.close(); } - @Test - public void testNonceCollision() throws Exception { - LOG.info("test=testNonceCollision"); - try ( - ConnectionImplementation connection = - ConnectionFactory.createConnectionImpl(UTIL.getConfiguration(), null, - UserProvider.instantiate(UTIL.getConfiguration()).getCurrent()); - Table table = connection.getTable(TEST_TABLE)) { - Put put = new Put(ONE_ROW); - put.addColumn(BYTES_FAMILY, QUALIFIER, Bytes.toBytes(0L)); - - // Replace nonce manager with the one that returns each nonce twice. - NonceGenerator cnm = new NonceGenerator() { - - private final PerClientRandomNonceGenerator delegate = PerClientRandomNonceGenerator.get(); - - private long lastNonce = -1; - - @Override - public synchronized long newNonce() { - long nonce = 0; - if (lastNonce == -1) { - nonce = delegate.newNonce(); - lastNonce = nonce; - } else { - nonce = lastNonce; - lastNonce = -1L; - } - return nonce; - } - - @Override - public long getNonceGroup() { - return delegate.getNonceGroup(); - } - }; - - NonceGenerator oldCnm = - ConnectionUtils.injectNonceGeneratorForTesting((ConnectionImplementation) connection, cnm); - - // First test sequential requests. - Increment inc = new Increment(ONE_ROW); - inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L); - table.increment(inc); - - // duplicate increment - inc = new Increment(ONE_ROW); - inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L); - Result result = table.increment(inc); - validateResult(result, QUALIFIER, Bytes.toBytes(1L)); - - Get get = new Get(ONE_ROW); - get.addColumn(BYTES_FAMILY, QUALIFIER); - result = table.get(get); - validateResult(result, QUALIFIER, Bytes.toBytes(1L)); - - // Now run a bunch of requests in parallel, exactly half should succeed. - int numRequests = 40; - final CountDownLatch startedLatch = new CountDownLatch(numRequests); - final CountDownLatch startLatch = new CountDownLatch(1); - final CountDownLatch doneLatch = new CountDownLatch(numRequests); - for (int i = 0; i < numRequests; ++i) { - Runnable r = new Runnable() { - @Override - public void run() { - Table table = null; - try { - table = connection.getTable(TEST_TABLE); - } catch (IOException e) { - fail("Not expected"); - } - Increment inc = new Increment(ONE_ROW); - inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L); - startedLatch.countDown(); - try { - startLatch.await(); - } catch (InterruptedException e) { - fail("Not expected"); - } - try { - table.increment(inc); - } catch (IOException ioEx) { - fail("Not expected"); - } - doneLatch.countDown(); - } - }; - Threads.setDaemonThreadRunning(new Thread(r)); - } - startedLatch.await(); // Wait until all threads are ready... - startLatch.countDown(); // ...and unleash the herd! - doneLatch.await(); - // Now verify - get = new Get(ONE_ROW); - get.addColumn(BYTES_FAMILY, QUALIFIER); - result = table.get(get); - validateResult(result, QUALIFIER, Bytes.toBytes((numRequests / 2) + 1L)); - } - } - @Test public void testBatchWithMixedActions() throws Exception { LOG.info("test=testBatchWithMixedActions"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocationCaching.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocationCaching.java index 44efcbc59564..276023c9b2d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocationCaching.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocationCaching.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.client; import static org.junit.Assert.assertNotEquals; @@ -33,14 +32,9 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -/** - * To be rewrite to check async meta cache. - */ -@Ignore @Category({MediumTests.class, ClientTests.class}) public class TestRegionLocationCaching { @@ -109,7 +103,8 @@ public void testCachingForHTableMultiPut() throws Exception { private void checkRegionLocationIsCached(final TableName tableName, final Connection conn) throws InterruptedException, IOException { for (int count = 0; count < 50; count++) { - int number = ((ConnectionImplementation) conn).getNumberOfCachedRegionLocations(tableName); + int number = ((AsyncConnectionImpl) conn.toAsyncConnection()).getLocator() + .getNumberOfCachedRegionLocations(tableName); assertNotEquals("Expected non-zero number of cached region locations", 0, number); Thread.sleep(100); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 61d4b86a4353..52241f819fcd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; @@ -60,7 +58,6 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -640,143 +637,4 @@ public void testReplicaGetWithRpcClientImpl() throws IOException { HTU.deleteTable(hdt.getTableName()); } } - - // This test is to test when hbase.client.metaReplicaCallTimeout.scan is configured, meta table - // scan will always get the result from primary meta region as long as the result is returned - // within configured hbase.client.metaReplicaCallTimeout.scan from primary meta region. - // To be rewrite, and meta replicas is not stable - @Ignore - @Test - public void testGetRegionLocationFromPrimaryMetaRegion() throws IOException, InterruptedException { - HTU.getAdmin().balancerSwitch(false, true); - - ((ConnectionImplementation) HTU.getConnection()).setUseMetaReplicas(true); - - // Create table then get the single region for our new table. - HTableDescriptor hdt = HTU.createTableDescriptor("testGetRegionLocationFromPrimaryMetaRegion"); - hdt.setRegionReplication(2); - try { - - HTU.createTable(hdt, new byte[][] { f }, null); - - RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = true; - - // Get user table location, always get it from the primary meta replica - try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) { - locator.getRegionLocations(row, true); - } - } finally { - RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = false; - ((ConnectionImplementation) HTU.getConnection()).setUseMetaReplicas(false); - HTU.getAdmin().balancerSwitch(true, true); - HTU.getAdmin().disableTable(hdt.getTableName()); - HTU.deleteTable(hdt.getTableName()); - } - } - - - // This test is to simulate the case that the meta region and the primary user region - // are down, hbase client is able to access user replica regions and return stale data. - // Meta replica is enabled to show the case that the meta replica region could be out of sync - // with the primary meta region. - // To be rewrite, and meta replicas is not stable - @Ignore - @Test - public void testReplicaGetWithPrimaryAndMetaDown() throws IOException, InterruptedException { - HTU.getAdmin().balancerSwitch(false, true); - - ((ConnectionImplementation)HTU.getConnection()).setUseMetaReplicas(true); - - // Create table then get the single region for our new table. - HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaGetWithPrimaryAndMetaDown"); - hdt.setRegionReplication(2); - try { - Table table = HTU.createTable(hdt, new byte[][] { f }, null); - // Get Meta location - RegionLocations mrl; - try ( - RegionLocator locator = HTU.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) { - mrl = new RegionLocations(locator.getRegionLocations(HConstants.EMPTY_START_ROW, true)); - } - - // Get user table location - RegionLocations url; - try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) { - url = new RegionLocations(locator.getRegionLocations(row, true)); - } - - // Make sure that user primary region is co-hosted with the meta region - if (!url.getDefaultRegionLocation().getServerName().equals( - mrl.getDefaultRegionLocation().getServerName())) { - HTU.moveRegionAndWait(url.getDefaultRegionLocation().getRegionInfo(), - mrl.getDefaultRegionLocation().getServerName()); - } - - // Make sure that the user replica region is not hosted by the same region server with - // primary - if (url.getRegionLocation(1).getServerName().equals(mrl.getDefaultRegionLocation() - .getServerName())) { - HTU.moveRegionAndWait(url.getRegionLocation(1).getRegionInfo(), - url.getDefaultRegionLocation().getServerName()); - } - - // Wait until the meta table is updated with new location info - while (true) { - try (RegionLocator locator = - HTU.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) { - mrl = new RegionLocations(locator.getRegionLocations(HConstants.EMPTY_START_ROW, true)); - } - - // Get user table location - try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) { - url = new RegionLocations(locator.getRegionLocations(row, true)); - } - - LOG.info("meta locations " + mrl); - LOG.info("table locations " + url); - ServerName a = url.getDefaultRegionLocation().getServerName(); - ServerName b = mrl.getDefaultRegionLocation().getServerName(); - if(a.equals(b)) { - break; - } else { - LOG.info("Waiting for new region info to be updated in meta table"); - Thread.sleep(100); - } - } - - Put p = new Put(row); - p.addColumn(f, row, row); - table.put(p); - - // Flush so it can be picked by the replica refresher thread - HTU.flush(table.getName()); - - // Sleep for some time until data is picked up by replicas - try { - Thread.sleep(2 * REFRESH_PERIOD); - } catch (InterruptedException e1) { - LOG.error(e1.toString(), e1); - } - - // Simulating the RS down - RegionServerHostingPrimayMetaRegionSlowOrStopCopro.throwException = true; - - // The first Get is supposed to succeed - Get g = new Get(row); - g.setConsistency(Consistency.TIMELINE); - Result r = table.get(g); - Assert.assertTrue(r.isStale()); - - // The second Get will succeed as well - r = table.get(g); - Assert.assertTrue(r.isStale()); - - } finally { - ((ConnectionImplementation)HTU.getConnection()).setUseMetaReplicas(false); - RegionServerHostingPrimayMetaRegionSlowOrStopCopro.throwException = false; - HTU.getAdmin().balancerSwitch(true, true); - HTU.getAdmin().disableTable(hdt.getTableName()); - HTU.deleteTable(hdt.getTableName()); - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index befe28bebb96..ec6f130e8485 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -19,13 +19,9 @@ import com.codahale.metrics.Counter; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Optional; import java.util.Random; -import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -38,10 +34,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.NotServingRegionException; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -52,7 +47,6 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster; -import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -63,7 +57,6 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -199,7 +192,6 @@ public static void beforeClass() throws Exception { StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, REFRESH_PERIOD); HTU.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true); HTU.getConfiguration().setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true); - ConnectionUtils.setupMasterlessConnection(HTU.getConfiguration()); HTU.startMiniCluster(NB_SERVERS); // Create table then get the single region for our new table. @@ -332,21 +324,20 @@ public void testLocations() throws Exception { byte[] b1 = Bytes.toBytes("testLocations"); openRegion(hriSecondary); - try ( - ConnectionImplementation hc = ConnectionFactory.createConnectionImpl(HTU.getConfiguration(), - null, UserProvider.instantiate(HTU.getConfiguration()).getCurrent())) { - hc.clearRegionLocationCache(); - RegionLocations rl = hc.locateRegion(table.getName(), b1, false, false); + try (Connection conn = ConnectionFactory.createConnection(HTU.getConfiguration()); + RegionLocator locator = conn.getRegionLocator(TABLE_NAME)) { + conn.clearRegionLocationCache(); + List rl = locator.getRegionLocations(b1, true); Assert.assertEquals(2, rl.size()); - rl = hc.locateRegion(table.getName(), b1, true, false); + rl = locator.getRegionLocations(b1, false); Assert.assertEquals(2, rl.size()); - hc.clearRegionLocationCache(); - rl = hc.locateRegion(table.getName(), b1, true, false); + conn.clearRegionLocationCache(); + rl = locator.getRegionLocations(b1, false); Assert.assertEquals(2, rl.size()); - rl = hc.locateRegion(table.getName(), b1, false, false); + rl = locator.getRegionLocations(b1, true); Assert.assertEquals(2, rl.size()); } finally { closeRegion(hriSecondary); @@ -555,10 +546,6 @@ public void testUseRegionWithReplica() throws Exception { } } - /** - * To be rewrite without ConnectionImplementation - */ - @Ignore @Test public void testHedgedRead() throws Exception { byte[] b1 = Bytes.toBytes("testHedgedRead"); @@ -579,16 +566,18 @@ public void testHedgedRead() throws Exception { LOG.info("get works and is not stale done"); //reset - ConnectionImplementation connection = (ConnectionImplementation) HTU.getConnection(); - Counter hedgedReadOps = connection.getConnectionMetrics().hedgedReadOps; - Counter hedgedReadWin = connection.getConnectionMetrics().hedgedReadWin; + AsyncConnectionImpl conn = (AsyncConnectionImpl) HTU.getConnection().toAsyncConnection(); + Counter hedgedReadOps = conn.getConnectionMetrics().get().hedgedReadOps; + Counter hedgedReadWin = conn.getConnectionMetrics().get().hedgedReadWin; hedgedReadOps.dec(hedgedReadOps.getCount()); hedgedReadWin.dec(hedgedReadWin.getCount()); // Wait a little on the main region, just enough to happen once hedged read // and hedged read did not returned faster - int primaryCallTimeoutMicroSecond = connection.getConnectionConfiguration().getPrimaryCallTimeoutMicroSecond(); - SlowMeCopro.sleepTime.set(TimeUnit.MICROSECONDS.toMillis(primaryCallTimeoutMicroSecond)); + long primaryCallTimeoutNs = conn.connConf.getPrimaryCallTimeoutNs(); + // The resolution of our timer is 10ms, so we need to sleep a bit more otherwise we may not + // trigger the hedged read... + SlowMeCopro.sleepTime.set(TimeUnit.NANOSECONDS.toMillis(primaryCallTimeoutNs) + 100); SlowMeCopro.getSecondaryCdl().set(new CountDownLatch(1)); g = new Get(b1); g.setConsistency(Consistency.TIMELINE); @@ -610,7 +599,9 @@ public void testHedgedRead() throws Exception { Assert.assertTrue(r.isStale()); Assert.assertTrue(r.getColumnCells(f, b1).isEmpty()); Assert.assertEquals(2, hedgedReadOps.getCount()); - Assert.assertEquals(1, hedgedReadWin.getCount()); + // we update the metrics after we finish the request so we use a waitFor here, use assert + // directly may cause failure if we run too fast. + HTU.waitFor(10000, () -> hedgedReadWin.getCount() == 1); SlowMeCopro.getPrimaryCdl().get().countDown(); LOG.info("hedged read occurred and faster"); @@ -623,299 +614,4 @@ public void testHedgedRead() throws Exception { closeRegion(hriSecondary); } } - - @Ignore // Disabled because it is flakey. Fails 17% on constrained GCE. %3 on Apache. - @Test - public void testCancelOfMultiGet() throws Exception { - openRegion(hriSecondary); - try { - List puts = new ArrayList<>(2); - byte[] b1 = Bytes.toBytes("testCancelOfMultiGet" + 0); - Put p = new Put(b1); - p.addColumn(f, b1, b1); - puts.add(p); - - byte[] b2 = Bytes.toBytes("testCancelOfMultiGet" + 1); - p = new Put(b2); - p.addColumn(f, b2, b2); - puts.add(p); - table.put(puts); - LOG.debug("PUT done"); - flushRegion(hriPrimary); - LOG.info("flush done"); - - Thread.sleep(1000 + REFRESH_PERIOD * 2); - - AsyncProcess ap = ((ConnectionImplementation) HTU.getConnection()).getAsyncProcess(); - - // Make primary slowdown - SlowMeCopro.getPrimaryCdl().set(new CountDownLatch(1)); - - List gets = new ArrayList<>(); - Get g = new Get(b1); - g.setCheckExistenceOnly(true); - g.setConsistency(Consistency.TIMELINE); - gets.add(g); - g = new Get(b2); - g.setCheckExistenceOnly(true); - g.setConsistency(Consistency.TIMELINE); - gets.add(g); - Object[] results = new Object[2]; - - int operationTimeout = ((ConnectionImplementation) HTU.getConnection()) - .getConnectionConfiguration().getOperationTimeout(); - int readTimeout = ((ConnectionImplementation) 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) { - Assert.assertTrue(((Result)r).isStale()); - Assert.assertTrue(((Result)r).getExists()); - } - Set set = - ((AsyncRequestFutureImpl)reqs).getCallsInProgress(); - // verify we did cancel unneeded calls - Assert.assertTrue(!set.isEmpty()); - for (CancellableRegionServerCallable m : set) { - Assert.assertTrue(m.isCancelled()); - } - } finally { - SlowMeCopro.getPrimaryCdl().get().countDown(); - SlowMeCopro.sleepTime.set(0); - SlowMeCopro.slowDownNext.set(false); - SlowMeCopro.countOfNext.set(0); - for (int i = 0; i < 2; i++) { - byte[] b1 = Bytes.toBytes("testCancelOfMultiGet" + i); - Delete d = new Delete(b1); - table.delete(d); - } - closeRegion(hriSecondary); - } - } - - /** - * To be rewrite - */ - @Ignore - @Test - public void testScanWithReplicas() throws Exception { - //simple scan - runMultipleScansOfOneType(false, false); - } - - /** - * To be rewrite - */ - @Ignore - @Test - public void testSmallScanWithReplicas() throws Exception { - //small scan - runMultipleScansOfOneType(false, true); - } - - /** - * To be rewrite - */ - @Ignore - @Test - public void testReverseScanWithReplicas() throws Exception { - //reverse scan - runMultipleScansOfOneType(true, false); - } - - /** - * To be rewrite - */ - @Ignore - @Test - public void testCancelOfScan() throws Exception { - openRegion(hriSecondary); - int NUMROWS = 100; - try { - for (int i = 0; i < NUMROWS; i++) { - byte[] b1 = Bytes.toBytes("testUseRegionWithReplica" + i); - Put p = new Put(b1); - p.addColumn(f, b1, b1); - table.put(p); - } - LOG.debug("PUT done"); - int caching = 20; - byte[] start; - start = Bytes.toBytes("testUseRegionWithReplica" + 0); - - flushRegion(hriPrimary); - LOG.info("flush done"); - Thread.sleep(1000 + REFRESH_PERIOD * 2); - - // now make some 'next' calls slow - SlowMeCopro.slowDownNext.set(true); - SlowMeCopro.countOfNext.set(0); - SlowMeCopro.sleepTime.set(5000); - - Scan scan = new Scan(start); - scan.setCaching(caching); - scan.setConsistency(Consistency.TIMELINE); - ResultScanner scanner = table.getScanner(scan); - Iterator iter = scanner.iterator(); - iter.next(); - Assert.assertTrue(((ClientScanner)scanner).isAnyRPCcancelled()); - SlowMeCopro.slowDownNext.set(false); - SlowMeCopro.countOfNext.set(0); - } finally { - SlowMeCopro.getPrimaryCdl().get().countDown(); - SlowMeCopro.sleepTime.set(0); - SlowMeCopro.slowDownNext.set(false); - SlowMeCopro.countOfNext.set(0); - for (int i = 0; i < NUMROWS; i++) { - byte[] b1 = Bytes.toBytes("testUseRegionWithReplica" + i); - Delete d = new Delete(b1); - table.delete(d); - } - closeRegion(hriSecondary); - } - } - - private void runMultipleScansOfOneType(boolean reversed, boolean small) throws Exception { - openRegion(hriSecondary); - int NUMROWS = 100; - int NUMCOLS = 10; - try { - for (int i = 0; i < NUMROWS; i++) { - byte[] b1 = Bytes.toBytes("testUseRegionWithReplica" + i); - for (int col = 0; col < NUMCOLS; col++) { - Put p = new Put(b1); - String qualifier = "qualifer" + col; - KeyValue kv = new KeyValue(b1, f, Bytes.toBytes(qualifier)); - p.add(kv); - table.put(p); - } - } - LOG.debug("PUT done"); - int caching = 20; - long maxResultSize = Long.MAX_VALUE; - - byte[] start; - if (reversed) start = Bytes.toBytes("testUseRegionWithReplica" + (NUMROWS - 1)); - else start = Bytes.toBytes("testUseRegionWithReplica" + 0); - - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, - start, NUMROWS, NUMCOLS, false, false); - - // Even if we were to slow the server down, unless we ask for stale - // we won't get it - SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, start, NUMROWS, - NUMCOLS, false, false); - SlowMeCopro.sleepTime.set(0); - - flushRegion(hriPrimary); - LOG.info("flush done"); - Thread.sleep(1000 + REFRESH_PERIOD * 2); - - //Now set the flag to get a response even if stale - SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, - start, NUMROWS, NUMCOLS, true, false); - SlowMeCopro.sleepTime.set(0); - - // now make some 'next' calls slow - SlowMeCopro.slowDownNext.set(true); - SlowMeCopro.countOfNext.set(0); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start, - NUMROWS, NUMCOLS, true, true); - SlowMeCopro.slowDownNext.set(false); - SlowMeCopro.countOfNext.set(0); - - // Make sure we do not get stale data.. - SlowMeCopro.sleepTime.set(5000); - scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, - start, NUMROWS, NUMCOLS, false, false); - SlowMeCopro.sleepTime.set(0); - - // While the next calls are slow, set maxResultSize to 1 so that some partial results will be - // returned from the server before the replica switch occurs. - maxResultSize = 1; - SlowMeCopro.slowDownNext.set(true); - SlowMeCopro.countOfNext.set(0); - scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start, - NUMROWS, NUMCOLS, true, true); - maxResultSize = Long.MAX_VALUE; - SlowMeCopro.slowDownNext.set(false); - SlowMeCopro.countOfNext.set(0); - } finally { - SlowMeCopro.getPrimaryCdl().get().countDown(); - SlowMeCopro.sleepTime.set(0); - SlowMeCopro.slowDownNext.set(false); - SlowMeCopro.countOfNext.set(0); - for (int i = 0; i < NUMROWS; i++) { - byte[] b1 = Bytes.toBytes("testUseRegionWithReplica" + i); - Delete d = new Delete(b1); - table.delete(d); - } - closeRegion(hriSecondary); - } - } - - private void scanWithReplicas(boolean reversed, boolean small, Consistency consistency, - int caching, long maxResultSize, byte[] startRow, int numRows, int numCols, - boolean staleExpected, boolean slowNext) - throws Exception { - Scan scan = new Scan(startRow); - scan.setCaching(caching); - scan.setMaxResultSize(maxResultSize); - scan.setReversed(reversed); - scan.setSmall(small); - scan.setConsistency(consistency); - ResultScanner scanner = table.getScanner(scan); - Iterator iter = scanner.iterator(); - - // Maps of row keys that we have seen so far - HashMap map = new HashMap<>(); - - // Tracked metrics - int rowCount = 0; - int cellCount = 0; - int countOfStale = 0; - - while (iter.hasNext()) { - rowCount++; - Result r = iter.next(); - String row = new String(r.getRow()); - - if (map.containsKey(row)) { - throw new Exception("Unexpected scan result. Repeated row " + Bytes.toString(r.getRow())); - } - - map.put(row, true); - - for (Cell cell : r.rawCells()) { - cellCount++; - } - - if (!slowNext) Assert.assertTrue(r.isStale() == staleExpected); - if (r.isStale()) countOfStale++; - } - Assert.assertTrue("Count of rows " + rowCount + " num rows expected " + numRows, - rowCount == numRows); - Assert.assertTrue("Count of cells: " + cellCount + " cells expected: " + numRows * numCols, - cellCount == (numRows * numCols)); - - if (slowNext) { - LOG.debug("Count of Stale " + countOfStale); - Assert.assertTrue(countOfStale > 1); - - // If the scan was configured in such a way that a full row was NOT retrieved before the - // replica switch occurred, then it is possible that all rows were stale - if (maxResultSize != Long.MAX_VALUE) { - Assert.assertTrue(countOfStale <= numRows); - } else { - Assert.assertTrue(countOfStale < numRows); - } - } - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java index 1a94af013b8d..9f26f3c96434 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java @@ -63,7 +63,7 @@ public static void setUpBeforeClass() throws Exception { conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS); // effectively limit max result size to one entry if it has tags conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, SCANNER_DATA_LIMIT); - conf.setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true); + conf.setBoolean(ConnectionConfiguration.LOG_SCANNER_ACTIVITY, true); TEST_UTIL.startMiniCluster(1); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index 4f03108b5dab..05afcdacd95e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -17,33 +17,17 @@ */ package org.apache.hadoop.hbase.master; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import java.io.IOException; -import java.net.InetAddress; -import java.net.UnknownHostException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.MetaMockingUtil; -import org.apache.hadoop.hbase.ServerLoad; -import org.apache.hadoop.hbase.ServerMetricsBuilder; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.HConnectionTestingUtility; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; @@ -53,17 +37,10 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; /** * Standup the master and fake it to test various aspects of master function. @@ -80,7 +57,6 @@ public class TestMasterNoCluster { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMasterNoCluster.class); - private static final Logger LOG = LoggerFactory.getLogger(TestMasterNoCluster.class); private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility(); @Rule @@ -123,9 +99,6 @@ public boolean isAborted() { /** * Test starting master then stopping it before its fully up. - * @throws IOException - * @throws KeeperException - * @throws InterruptedException */ @Test public void testStopDuringStart() @@ -137,174 +110,7 @@ public void testStopDuringStart() master.join(); } - /** - * Test master failover. - * Start up three fake regionservers and a master. - * @throws IOException - * @throws KeeperException - * @throws InterruptedException - * @throws org.apache.hbase.thirdparty.com.google.protobuf.ServiceException - */ - @Ignore @Test // Disabled since HBASE-18511. Reenable when master can carry regions. - public void testFailover() throws Exception { - final long now = System.currentTimeMillis(); - // Names for our three servers. Make the port numbers match hostname. - // Will come in use down in the server when we need to figure how to respond. - final ServerName sn0 = ServerName.valueOf("0.example.org", 0, now); - final ServerName sn1 = ServerName.valueOf("1.example.org", 1, now); - final ServerName sn2 = ServerName.valueOf("2.example.org", 2, now); - final ServerName [] sns = new ServerName [] {sn0, sn1, sn2}; - // Put up the mock servers - final Configuration conf = TESTUTIL.getConfiguration(); - final MockRegionServer rs0 = new MockRegionServer(conf, sn0); - final MockRegionServer rs1 = new MockRegionServer(conf, sn1); - final MockRegionServer rs2 = new MockRegionServer(conf, sn2); - // Put some data into the servers. Make it look like sn0 has the metaH - // Put data into sn2 so it looks like it has a few regions for a table named 't'. - MetaTableLocator.setMetaLocation(rs0.getZooKeeper(), - rs0.getServerName(), RegionState.State.OPEN); - final TableName tableName = TableName.valueOf(name.getMethodName()); - Result [] results = new Result [] { - MetaMockingUtil.getMetaTableRowResult( - new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HBaseTestingUtility.KEYS[1]), - rs2.getServerName()), - MetaMockingUtil.getMetaTableRowResult( - new HRegionInfo(tableName, HBaseTestingUtility.KEYS[1], HBaseTestingUtility.KEYS[2]), - rs2.getServerName()), - MetaMockingUtil.getMetaTableRowResult(new HRegionInfo(tableName, HBaseTestingUtility.KEYS[2], - HConstants.EMPTY_END_ROW), - rs2.getServerName()) - }; - rs1.setNextResults(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), results); - - // Create master. Subclass to override a few methods so we can insert mocks - // and get notification on transitions. We need to fake out any rpcs the - // master does opening/closing regions. Also need to fake out the address - // of the 'remote' mocked up regionservers. - // Insert a mock for the connection, use TESTUTIL.getConfiguration rather than - // the conf from the master; the conf will already have an ClusterConnection - // associate so the below mocking of a connection will fail. - final Connection mockedConnection = HConnectionTestingUtility.getMockedConnectionAndDecorate( - TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(), - HRegionInfo.FIRST_META_REGIONINFO); - HMaster master = new HMaster(conf) { - @Override - InetAddress getRemoteInetAddress(final int port, final long serverStartCode) - throws UnknownHostException { - // Return different address dependent on port passed. - if (port > sns.length) { - return super.getRemoteInetAddress(port, serverStartCode); - } - ServerName sn = sns[port]; - return InetAddress.getByAddress(sn.getHostname(), - new byte [] {10, 0, 0, (byte)sn.getPort()}); - } - - @Override - protected void initClusterSchemaService() throws IOException, InterruptedException {} - - @Override - protected ServerManager createServerManager(MasterServices master) throws IOException { - ServerManager sm = super.createServerManager(master); - // Spy on the created servermanager - ServerManager spy = Mockito.spy(sm); - return spy; - } - - @Override - public Connection getConnection() { - return mockedConnection; - } - }; - master.start(); - - try { - // Wait till master is up ready for RPCs. - while (!master.serviceStarted) Threads.sleep(10); - // Fake master that there are regionservers out there. Report in. - for (int i = 0; i < sns.length; i++) { - RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder(); - ServerName sn = ServerName.parseVersionedServerName(sns[i].getVersionedBytes()); - request.setServer(ProtobufUtil.toServerName(sn)); - request.setLoad(ServerMetricsBuilder.toServerLoad(ServerMetricsBuilder.of(sn))); - master.getMasterRpcServices().regionServerReport(null, request.build()); - } - // Master should now come up. - while (!master.isInitialized()) { - Threads.sleep(100); - } - assertTrue(master.isInitialized()); - } finally { - rs0.stop("Test is done"); - rs1.stop("Test is done"); - rs2.stop("Test is done"); - master.stopMaster(); - master.join(); - } - } - - @Ignore @Test // Disabled since HBASE-18511. Reenable when master can carry regions. - public void testNotPullingDeadRegionServerFromZK() - throws IOException, KeeperException, InterruptedException { - final Configuration conf = TESTUTIL.getConfiguration(); - final ServerName newServer = ServerName.valueOf("test.sample", 1, 101); - final ServerName deadServer = ServerName.valueOf("test.sample", 1, 100); - final MockRegionServer rs0 = new MockRegionServer(conf, newServer); - - HMaster master = new HMaster(conf) { - @Override - protected MasterMetaBootstrap createMetaBootstrap() { - return new MasterMetaBootstrap(this) { - @Override - protected void assignMetaReplicas() - throws IOException, InterruptedException, KeeperException { - // Nothing to do. - } - }; - } - - @Override - protected void initClusterSchemaService() throws IOException, InterruptedException {} - - @Override - protected void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, - KeeperException, ReplicationException { - super.initializeZKBasedSystemTrackers(); - // Record a newer server in server manager at first - getServerManager().recordNewServerWithLock(newServer, - new ServerLoad(ServerMetricsBuilder.of(newServer))); - } - - @Override - public Connection getConnection() { - // Insert a mock for the connection, use TESTUTIL.getConfiguration rather than - // the conf from the master; the conf will already have a Connection - // associate so the below mocking of a connection will fail. - try { - return HConnectionTestingUtility.getMockedConnectionAndDecorate( - TESTUTIL.getConfiguration(), rs0, rs0, rs0.getServerName(), - HRegionInfo.FIRST_META_REGIONINFO); - } catch (IOException e) { - return null; - } - } - }; - master.start(); - - try { - // Wait till master is initialized. - while (!master.isInitialized()) Threads.sleep(10); - LOG.info("Master is initialized"); - - assertFalse("The dead server should not be pulled in", - master.getServerManager().isServerOnline(deadServer)); - } finally { - master.stopMaster(); - master.join(); - } - } - - @Test(timeout = 60000) + @Test public void testMasterInitWithSameClientServerZKQuorum() throws Exception { Configuration conf = new Configuration(TESTUTIL.getConfiguration()); conf.set(HConstants.CLIENT_ZOOKEEPER_QUORUM, HConstants.LOCALHOST); @@ -315,7 +121,7 @@ public void testMasterInitWithSameClientServerZKQuorum() throws Exception { master.join(); } - @Test(timeout = 60000) + @Test public void testMasterInitWithObserverModeClientZKQuorum() throws Exception { Configuration conf = new Configuration(TESTUTIL.getConfiguration()); Assert.assertFalse(Boolean.getBoolean(HConstants.CLIENT_ZOOKEEPER_OBSERVER_MODE)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java index ef64c940c3bb..52b4b716310d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; -import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; @@ -141,13 +140,7 @@ public MultiResponse answer(InvocationOnMock invocation) throws Throwable { } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); } - // Mock n ClusterConnection and an AdminProtocol implementation. Have the - // ClusterConnection return the HRI. Have the HRI return a few mocked up responses - // to make our test work. - this.connection = - HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(), - Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri, MOCK_MASTER_SERVERNAME, - RegionInfoBuilder.FIRST_META_REGIONINFO); + this.connection = HConnectionTestingUtility.getMockedConnection(getConfiguration()); // Set hbase.rootdir into test dir. Path rootdir = FSUtils.getRootDir(getConfiguration()); FSUtils.setRootDir(getConfiguration(), rootdir); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java index af8cfb8df3ba..847b5b7560e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java @@ -52,8 +52,6 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.RpcRetryingCaller; -import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -212,8 +210,6 @@ public void doAnAction() throws Exception { final Connection conn = UTIL.getConnection(); // Periodically do compaction to reduce the number of open file handles. if (numBulkLoads.get() % 5 == 0) { - RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf); - RpcRetryingCaller caller = factory. newCaller(); // 5 * 50 = 250 open file handles! try (RegionLocator locator = conn.getRegionLocator(tableName)) { HRegionLocation loc = locator.getRegionLocation(Bytes.toBytes("aaa"), true);