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 extends Row> 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 extends Row> 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 extends Row> 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 extends Row> 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 extends Row> 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 extends Row> rows) {
- this.rows = new ListRowAccess<>(rows);
- return this;
- }
-
- public Builder setRowAccess(RowAccess extends Row> 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 extends Row> 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 extends Row> 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 extends Row> 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 extends Row> 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 extends Runnable> 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 extends Runnable> 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 extends Row> 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 extends Mutation> 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 extends ClusterStatusListener.Listener> 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